You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "coufon (via GitHub)" <gi...@apache.org> on 2023/04/23 06:25:29 UTC

[GitHub] [iceberg] coufon opened a new pull request, #7412: Add Iceberg Catalog for GCP BigLake Metastore

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

   Add the basic implementation of a new Iceberg catalog for GCP BigLake Metastore.
   
   BigLake Metastore (BLMS) is a serverless metastore for Dataproc and BigQuery on GCP. BLMS provides a HMS compatible API for Iceberg tables. Iceberg tables stored in BLMS are queryable in BigQuery (https://cloud.google.com/bigquery/docs/iceberg-tables).
   
   BLMS API reference: https://cloud.google.com/bigquery/docs/reference/biglake/rest
   BLMS API clients: https://github.com/googleapis/google-cloud-java/tree/main/java-biglake
   


-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1266933778


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.ArgumentCaptor;
+
+public class BigLakeTableOperationsTest {
+
+  @TempDir private Path temp;
+
+  private static final String CATALOG_NAME = "iceberg";
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  private static final Schema SCHEMA =
+      new Schema(
+          required(1, "id", Types.IntegerType.get(), "unique ID"),
+          required(2, "data", Types.StringType.get()));
+
+  private BigLakeClient bigLakeClient = mock(BigLakeClient.class);
+  private BigLakeCatalog bigLakeCatalog;
+  private BigLakeTableOperations tableOps;
+
+  @BeforeEach
+  public void before() throws Exception {

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] djouallah commented on pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "djouallah (via GitHub)" <gi...@apache.org>.
djouallah commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1521043716

   sorry for posting here, ( asking from a user perspective)
   does this catalog support reading and writing iceberg table using third party tools ?
   is it compatible with Iceberg REST catalog, can I use it with pyiceberg ? 


-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1526807337

   @coufon can you help explain and document how the atomic update works?  I assume this is somehow related to the `etag` and `updateTable` call, but it's not entirely clear how the atomic swap is enforced.


-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1180614397


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "gcp_location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "blms_catalog";
+
+  public static final String HIVE_METASTORE_WAREHOUSE_DIR = "hive.metastore.warehouse.dir";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileSystem fs;
+  private FileIO fileIO;
+  private Configuration conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    if (!properties.containsKey(PROPERTIES_KEY_GCP_PROJECT)) {
+      throw new ValidationException("GCP project must be specified");
+    }
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+    BigLakeClient newClient;
+    try {
+      newClient =
+          new BigLakeClientImpl(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    if (this.conf == null) {
+      LOG.warn("No Hadoop Configuration was set, using the default environment Configuration");
+      this.conf = new Configuration();
+    }
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) {
+      this.conf.set(
+          HIVE_METASTORE_WAREHOUSE_DIR,
+          LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION)));
+    }
+
+    this.fs =
+        Util.getFs(
+            new Path(
+                LocationUtil.stripTrailingSlash(
+                    properties.get(CatalogProperties.WAREHOUSE_LOCATION))),
+            conf);
+
+    String fileIOImpl =
+        properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.hadoop.HadoopFileIO");

Review Comment:
   Updated to use 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1180611269


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";

Review Comment:
   It makes sense. I added a TODO to use GCPProperties in a following PR. I changed the config name to follow the existing style (e.g., biglake.project-id) in this PR. The issue of simply using GCPProperties in this PR is that "gcs.project-id" is too GCS specific, but the class field name is "projectId". I hope to change it to gcsProjectId, and add a bigLakeProjectId. It touches other classes so I prefer to separate it out.
   
   I actually feel "gcs.project-id" is not necessary: it would be great if customers can use buckets from any project (and different projects) in the same catalog handler. Most GCP customers today use a GCS connector (https://cloud.google.com/dataproc/docs/concepts/connectors/cloud-storage). It is an adapter between HDFS and GCS. A URL like "gs://bucket/folder" will be handled by this connector, project ID is detected from the bucket. It is open-source and pre-installed in Dataproc.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1186928842


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      Preconditions.checkArgument(
+          hiveOptions.containsParameters(METADATA_LOCATION_PROP),
+          "Table %s is not a valid Iceberg table, metadata location not found",
+          tableName());
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  private void createTable(String newMetadataLocation, TableMetadata metadata) {
+    LOG.debug("Creating a new Iceberg table: {}", tableName());
+    client.createTable(tableName, makeNewTable(metadata, newMetadataLocation));
+  }
+
+  /** Update table properties with concurrent update detection using etag. */
+  private void updateTable(
+      String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) {
+    Table table = client.getTable(tableName);
+    String etag = table.getEtag();
+    Preconditions.checkArgument(
+        !etag.isEmpty(),
+        "Etag of legacy table %s is empty, manually update the table by BigLake API or recreate and retry",
+        tableName());
+    HiveTableOptions options = table.getHiveOptions();
+
+    // If `metadataLocationFromMetastore` is different from metadata location of base, it means
+    // someone has updated metadata location in metastore, which is a conflict update.
+    String metadataLocationFromMetastore =
+        options.getParametersOrDefault(METADATA_LOCATION_PROP, "");
+    if (!metadataLocationFromMetastore.isEmpty()
+        && !metadataLocationFromMetastore.equals(oldMetadataLocation)) {
+      throw new CommitFailedException(
+          "Base metadata location '%s' is not same as the current table metadata location '%s' for"
+              + " %s.%s",
+          oldMetadataLocation,
+          metadataLocationFromMetastore,
+          tableName.getDatabase(),
+          tableName.getTable());
+    }
+
+    try {
+      // Updating a BLMS table with etag. The BLMS server transactionally (1) checks that the etag
+      // of a table on server is the same as the etag provided by the client, and (2) updates the
+      // table (and its etag). The server returns an error containing message "etag mismatch", if
+      // the etag on server has changed.
+      client.updateTableParameters(
+          tableName, buildTableParameters(newMetadataLocation, metadata), etag);
+    } catch (AbortedException e) {
+      if (e.getMessage().toLowerCase().contains("etag mismatch")) {
+        throw new CommitFailedException(
+            "Updating table failed due to conflict updates (etag mismatch)");
+      }

Review Comment:
   This needs to handle the `else` case. Otherwise the commit will appear to have succeeded and will likely result in data loss.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260161257


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.

Review Comment:
   It is optional. It defaults to the Spark plugin name. Updated 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] rdblue commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253621859


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new NoSuchIcebergTableException(
+            "Table %s is not a valid Iceberg table, metadata location not found", tableName());

Review Comment:
   `Invalid Iceberg table: missing metadata location`



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264586018


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              projectId,
+              region);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      bigLakeClient.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return metadata(loadDatabase(namespace));
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return bigLakeProperties == null ? ImmutableMap.of() : bigLakeProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    closeableGroup.close();

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264586209


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264587223


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))
+        .isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {}), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {"db", "tbl"}), ImmutableMap.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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1266091892


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.ArgumentCaptor;
+
+public class BigLakeTableOperationsTest {
+
+  @TempDir private Path temp;
+
+  private static final String CATALOG_NAME = "iceberg";
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  private static final Schema SCHEMA =
+      new Schema(
+          required(1, "id", Types.IntegerType.get(), "unique ID"),
+          required(2, "data", Types.StringType.get()));
+
+  private BigLakeClient bigLakeClient = mock(BigLakeClient.class);
+  private BigLakeCatalog bigLakeCatalog;
+  private BigLakeTableOperations tableOps;
+
+  @BeforeEach
+  public void before() throws Exception {
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.PROJECT_ID,
+            GCP_PROJECT,
+            GCPProperties.REGION,
+            GCP_REGION,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            temp.toFile().getAbsolutePath(),
+            GCPProperties.BIGLAKE_CATALOG_ID,
+            CATALOG_ID);
+
+    bigLakeCatalog = new BigLakeCatalog();
+    bigLakeCatalog.setConf(new Configuration());
+    bigLakeCatalog.initialize(CATALOG_NAME, properties, bigLakeClient);
+    tableOps = (BigLakeTableOperations) bigLakeCatalog.newTableOps(TABLE_IDENTIFIER);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalog.close();
+  }
+
+  @Test
+  public void testDoCommitShouldUseEtagForUpdateTable() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    Table createdTable = createTestTable();
+
+    Table tableWithEtag = createdTable.toBuilder().setEtag("etag").build();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(tableWithEtag, tableWithEtag);
+
+    org.apache.iceberg.Table loadedTable = bigLakeCatalog.loadTable(TABLE_IDENTIFIER);
+
+    when(bigLakeClient.updateTableParameters(any(), any(), any())).thenReturn(tableWithEtag);
+    loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit();
+
+    ArgumentCaptor<TableName> nameCaptor = ArgumentCaptor.forClass(TableName.class);
+    ArgumentCaptor<String> etagCaptor = ArgumentCaptor.forClass(String.class);
+    verify(bigLakeClient, times(1))
+        .updateTableParameters(nameCaptor.capture(), any(), etagCaptor.capture());
+    assertThat(nameCaptor.getValue()).isEqualTo(TABLE_NAME);
+    assertThat(etagCaptor.getValue()).isEqualTo("etag");
+  }
+
+  @Test
+  public void testDoCommitShouldFailWhenEtagMismatch() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    Table createdTable = createTestTable();
+
+    Table tableWithEtag = createdTable.toBuilder().setEtag("etag").build();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(tableWithEtag, tableWithEtag);
+
+    org.apache.iceberg.Table loadedTable = bigLakeCatalog.loadTable(TABLE_IDENTIFIER);
+
+    when(bigLakeClient.updateTableParameters(any(), any(), any()))
+        .thenThrow(
+            new AbortedException(
+                new RuntimeException("error message etag mismatch"),
+                GrpcStatusCode.of(Code.ABORTED),
+                false));
+
+    assertThatThrownBy(
+            () -> loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit())
+        .isInstanceOf(CommitFailedException.class);
+  }
+
+  @Test
+  public void testDoFreshRefreshShouldReturnNullForNonIcebergTable() {

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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344491768


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        String dbId = databaseId(namespace);
+        validateDatabaseId(dbId, namespace);
+        client.deleteDatabase(databaseName(dbId));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> props) {
+    String dbId = databaseId(namespace);
+    validateDatabaseId(dbId, namespace);
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(dbId).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(databaseName(dbId), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> props) {
+    String dbId = databaseId(namespace);
+    validateDatabaseId(dbId, namespace);
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(dbId).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(databaseName(dbId), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls catalog to check existence. BLMS catalog has no metadata today.
+      client.catalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      return loadDatabase(dbId).getHiveOptions().getParametersMap();

Review Comment:
   It is unmodifiable: https://protobuf.dev/reference/java/java-generated/#map-fields



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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344513347


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Catalog already exists: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Catalog catalog(CatalogName name) {
+    try {
+      return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+    } catch (PermissionDeniedException e) {
+      throw new NoSuchNamespaceException(
+          e, "Catalog does not exist: %s (or permission denied)", name.getCatalog());
+    }
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    try {
+      stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+    } catch (PermissionDeniedException e) {
+      throw new NoSuchNamespaceException(
+          e, "Catalog does not exist: %s (or permission denied)", name.getCatalog());
+    }
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {

Review Comment:
   Yes, it is a good point. We should check whether it failed due to parent not found or permission denied. It needs parsing the error message. I added a TODO to do it in a follow-up PR, to avoid adding more new code to this PR which has been around for long.
   
   The error message is like this (this example creates database): "Permission 'biglake.databases.create' denied on resource '//biglake.googleapis.com/projects/myproj/locations/us/catalogs/mycat' (or it may not exist).". We need to determine whether the error is on this resource (table) or its parent (database or catalog).



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

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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1750974517

   > Great work, this feature is exactly what my team needs. Are there any updates? Zhou Fang Ryan Blue
   
   We released these code here (https://cloud.google.com/bigquery/docs/manage-open-source-metadata#connect-dataproc-vm):
   
   Iceberg 1.2.0: gs://spark-lib/biglake/biglake-catalog-iceberg1.2.0-0.1.1-with-dependencies.jar
   Iceberg 0.14.0: gs://spark-lib/biglake/biglake-catalog-iceberg0.14.0-0.1.1-with-dependencies.jar
   
   Feel free to try these before this PR is merged.


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

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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "emkornfield (via GitHub)" <gi...@apache.org>.
emkornfield commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1773029673

   @rdblue just wanted to check if you had any remaining concerns 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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "istreeter (via GitHub)" <gi...@apache.org>.
istreeter commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-2027773396

   I just came across this PR -- it looks like this would be a hugely valuable addition to Iceberg!  My company would certainly benefit from this.
   
   It seems to have got stuck though.  Is there anything I can do to help get this over the finish line?  I'd be willing to help out if there's any chance this could get released.


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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-2068859607

   @coufon sorry for the delay here, this fell off the radar unfortunately. Could you rebase the PR please?


-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1180618458


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "gcp_location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "blms_catalog";
+
+  public static final String HIVE_METASTORE_WAREHOUSE_DIR = "hive.metastore.warehouse.dir";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileSystem fs;
+  private FileIO fileIO;
+  private Configuration conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    if (!properties.containsKey(PROPERTIES_KEY_GCP_PROJECT)) {
+      throw new ValidationException("GCP project must be specified");
+    }
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+    BigLakeClient newClient;
+    try {
+      newClient =
+          new BigLakeClientImpl(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    if (this.conf == null) {
+      LOG.warn("No Hadoop Configuration was set, using the default environment Configuration");
+      this.conf = new Configuration();
+    }
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) {
+      this.conf.set(
+          HIVE_METASTORE_WAREHOUSE_DIR,
+          LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION)));
+    }
+
+    this.fs =
+        Util.getFs(
+            new Path(
+                LocationUtil.stripTrailingSlash(
+                    properties.get(CatalogProperties.WAREHOUSE_LOCATION))),
+            conf);
+
+    String fileIOImpl =
+        properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.hadoop.HadoopFileIO");
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        conf,
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    client.deleteTable(
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    if (!fromDbId.equals(toDbId)) {
+      throw new ValidationException("New table name must be in the same database");
+    }
+
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {

Review Comment:
   Sorry for the confusion. There are two options: (1) link a `<spark-catalog>` to a physical `<biglake-catalog>`, so the full identifier of a table is just `<spark-catalog>.<database/schema>.<table>`. `CREATE/DROP <spark-catalog>` is supported: it creates/deletes `<biglake-catalog> `via API. (2) Use `<spark-catalog>.<biglake-catalog>.<database/schema>.<table>`.
   
   We choose (1) to avoid the long table identifier in (2) (linking is done by a config `biglake.catalog`). The limitation is that customers can't use two <biglake-catalog>s in the same <spark-catalog>, they have to install two <spark-catalog>s instead. We think it is OK, because <biglake-catalog> is usually an environmental level container (e.g., dev, staging, prod envs), cross env reference is rare.
   
   My concern is, I am not sure whether (1) violates any design pattern of Spark namespaces. Please let me know if it does.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1180618458


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "gcp_location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "blms_catalog";
+
+  public static final String HIVE_METASTORE_WAREHOUSE_DIR = "hive.metastore.warehouse.dir";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileSystem fs;
+  private FileIO fileIO;
+  private Configuration conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    if (!properties.containsKey(PROPERTIES_KEY_GCP_PROJECT)) {
+      throw new ValidationException("GCP project must be specified");
+    }
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+    BigLakeClient newClient;
+    try {
+      newClient =
+          new BigLakeClientImpl(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    if (this.conf == null) {
+      LOG.warn("No Hadoop Configuration was set, using the default environment Configuration");
+      this.conf = new Configuration();
+    }
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) {
+      this.conf.set(
+          HIVE_METASTORE_WAREHOUSE_DIR,
+          LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION)));
+    }
+
+    this.fs =
+        Util.getFs(
+            new Path(
+                LocationUtil.stripTrailingSlash(
+                    properties.get(CatalogProperties.WAREHOUSE_LOCATION))),
+            conf);
+
+    String fileIOImpl =
+        properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.hadoop.HadoopFileIO");
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        conf,
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    client.deleteTable(
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    if (!fromDbId.equals(toDbId)) {
+      throw new ValidationException("New table name must be in the same database");
+    }
+
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {

Review Comment:
   Sorry for the confusion. There are two options: (1) link a `<spark-catalog>` to a physical `<biglake-catalog>`, so the full identifier of a table is just `<spark-catalog>.<database/schema>.<table>`. `CREATE/DROP <spark-catalog>` is supported: it creates/deletes `<biglake-catalog> `via API. (2) Use `<spark-catalog>.<biglake-catalog>.<database/schema>.<table>`.
   
   We choose (1) to avoid the long table identifier in (2) (linking is done by a config `biglake.catalog`). The limitation is that customers can't use two `<biglake-catalog>`s in the same `<spark-catalog>`, they have to install two `<spark-catalog>`s instead. We think it is OK, because usually the tables to use together are in the same catalog.
   
   My concern is, I am not sure whether (1) violates any design pattern of Spark namespaces. Please let me know if it does.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1186929211


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      Preconditions.checkArgument(
+          hiveOptions.containsParameters(METADATA_LOCATION_PROP),
+          "Table %s is not a valid Iceberg table, metadata location not found",
+          tableName());
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  private void createTable(String newMetadataLocation, TableMetadata metadata) {
+    LOG.debug("Creating a new Iceberg table: {}", tableName());
+    client.createTable(tableName, makeNewTable(metadata, newMetadataLocation));
+  }
+
+  /** Update table properties with concurrent update detection using etag. */
+  private void updateTable(
+      String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) {
+    Table table = client.getTable(tableName);
+    String etag = table.getEtag();
+    Preconditions.checkArgument(
+        !etag.isEmpty(),
+        "Etag of legacy table %s is empty, manually update the table by BigLake API or recreate and retry",
+        tableName());
+    HiveTableOptions options = table.getHiveOptions();
+
+    // If `metadataLocationFromMetastore` is different from metadata location of base, it means
+    // someone has updated metadata location in metastore, which is a conflict update.
+    String metadataLocationFromMetastore =
+        options.getParametersOrDefault(METADATA_LOCATION_PROP, "");
+    if (!metadataLocationFromMetastore.isEmpty()
+        && !metadataLocationFromMetastore.equals(oldMetadataLocation)) {
+      throw new CommitFailedException(
+          "Base metadata location '%s' is not same as the current table metadata location '%s' for"
+              + " %s.%s",
+          oldMetadataLocation,
+          metadataLocationFromMetastore,
+          tableName.getDatabase(),
+          tableName.getTable());
+    }
+
+    try {
+      // Updating a BLMS table with etag. The BLMS server transactionally (1) checks that the etag
+      // of a table on server is the same as the etag provided by the client, and (2) updates the
+      // table (and its etag). The server returns an error containing message "etag mismatch", if
+      // the etag on server has changed.
+      client.updateTableParameters(
+          tableName, buildTableParameters(newMetadataLocation, metadata), etag);
+    } catch (AbortedException e) {
+      if (e.getMessage().toLowerCase().contains("etag mismatch")) {
+        throw new CommitFailedException(
+            "Updating table failed due to conflict updates (etag mismatch)");
+      }
+    }
+  }
+
+  private Table makeNewTable(TableMetadata metadata, String metadataFileLocation) {
+    Table.Builder builder = Table.newBuilder().setType(Table.Type.HIVE);

Review Comment:
   Would it be better to have an Iceberg type?



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264587043


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();

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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264755916


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)

Review Comment:
   What if `metadata` contains a `location` property?



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264585899


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO io;
+  // The catalog name.
+  private final String name;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO io, String name, TableName tableName) {
+    this.client = client;
+    this.io = io;
+    this.name = name;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new NoSuchIcebergTableException(
+            "Invalid Iceberg table %s: missing metadata location", tableName());
+      }
+
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (AlreadyExistsException | CommitFailedException | CommitStateUnknownException e) {

Review Comment:
   Good point. Updated 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268350945


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()),
+        name.getDatabase());
+  }
+
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getCatalog());
+  }
+
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table getTable(TableName name) {
+    if (name.getTable().isEmpty()) {
+      throw new NoSuchTableException("BigLake API does not allow tables with empty ID");
+    }
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", newName.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table deleteTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.deleteTable(
+                DeleteTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Iterable<Table> listTables(DatabaseName name) {
+    return convertException(
+        () ->
+            stub.listTables(ListTablesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getDatabase());
+  }
+
+  @Override
+  public void close() {
+    stub.close();
+  }
+
+  // Converts BigLake API errors to Iceberg errors.
+  private <T> T convertException(Supplier<T> result, String resourceId) {
+    try {
+      return result.get();
+    } catch (PermissionDeniedException e) {
+      throw new NotAuthorizedException(e, "BigLake API permission denied");
+    } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+      throw new AlreadyExistsException(e, "Namespace already exists: %s", resourceId);

Review Comment:
   We have 3 resources, table, catalog, database. "Table already exists" has already been caught locally, so it can only be catalog/database here, they are both namespace. I can move catching "com.google.api.gax.rpc.AlreadyExistsException" out but it seems verbose.



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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344550967


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        String dbId = databaseId(namespace);
+        validateDatabaseId(dbId, namespace);
+        client.deleteDatabase(databaseName(dbId));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);

Review Comment:
   I tried to return NotAuthorizedException from the client instead of 403. The problem is that the tests in CatalogTests expects NoSuchTableException and NoSuchNamespaceException. More refactoring is required. I feel convert 403 to 404 and make it is explicit to users is the best option 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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344472969


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());

Review Comment:
   Saw your other comment, I moved the throw code to caller method and removed the checkIdentifier and checkNamespace methods.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, 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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "devorbit (via GitHub)" <gi...@apache.org>.
devorbit commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1792967038

   Hi @dchristle @coufon 
   When are we expecting it to be available with standard iceberg-spark-runtime?
   We have some use cases for it to be used with the Iceberg Kafka Connector. I am not able to fetch this artifact from maven.
   
   Please guide me. Thanks.


-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1216926969


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new NoSuchIcebergTableException(
+            "Table %s is not a valid Iceberg table, metadata location not found", tableName());
+      }
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (AlreadyExistsException | CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }

Review Comment:
   Please update throughout for style. There should be whitespace between control flow blocks and the following statements.



-- 
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 pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1584858814

   Running CI


-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1192699919


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClientImpl.java:
##########
@@ -0,0 +1,287 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client implementation of Google BigLake service. */
+final class BigLakeClientImpl implements BigLakeClient {
+
+  private final String projectId;
+  private final String location;
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   * @param projectId GCP project ID
+   * @param location GCP region supported by BigLake, e.g., "us"
+   */
+  BigLakeClientImpl(String biglakeEndpoint, String projectId, String location) throws IOException {
+    this.projectId = projectId;
+    this.location = location;
+    this.stub =
+        MetastoreServiceClient.create(
+            MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  @Override
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()));
+  }
+
+  @Override
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Catalog %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Catalog %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()));
+  }
+
+  @Override
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Database %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Database %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll());
+  }
+
+  @Override
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Database %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () ->
+            stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build()));
+  }
+
+  @Override
+  public Table getTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Table deleteTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.deleteTable(
+                DeleteTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Iterable<Table> listTables(DatabaseName name) {
+    return convertException(
+        () ->
+            stub.listTables(ListTablesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll());
+  }
+
+  // Converts BigLake API errors to Iceberg errors.
+  private <T> T convertException(Supplier<T> result) {
+    try {
+      return result.get();
+    } catch (PermissionDeniedException e) {
+      throw new NotAuthorizedException(e, "Not authorized to call BigLake API");

Review Comment:
   Updated it to a more general message "BigLake API permission denied". The nested error `e` contains the detailed message from the server.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1192700182


##########
spark/v3.1/build.gradle:
##########
@@ -216,6 +216,7 @@ project(':iceberg-spark:iceberg-spark-runtime-3.1_2.12') {
     implementation (project(':iceberg-snowflake')) {
       exclude group: 'net.snowflake' , module: 'snowflake-jdbc'
     }
+    implementation project(':iceberg-gcp')

Review Comment:
   Sounds good. I removed the changes to spark build files.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268248307


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,404 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    return new BigLakeTableOperations(client, io, name(), tableName(db, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(db) : locationUri, identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else if (namespace.levels().length == 1) {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    } else {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(db, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    if (fromDb == null) {
+      throwInvalidDbNamespaceError(from.namespace());
+    }
+
+    String toDb = databaseId(to.namespace());
+    if (toDb == null) {
+      throwInvalidDbNamespaceError(to.namespace());
+    }
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> props) {
+    DatabaseName dbName = databaseName(namespace);
+    if (dbName == null) {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(dbName, optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> props) {
+    DatabaseName dbName = databaseName(namespace);
+    if (dbName == null) {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(dbName, optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return loadDatabase(namespace).getHiveOptions().getParametersMap();
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return properties == null ? ImmutableMap.of() : properties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closeableGroup != null) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    return String.format("%s/%s.db", warehouseLocation, dbId);
+  }
+
+  private static TableIdentifier tableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace namespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName tableName(String dbId, String tableId) {
+    return TableName.of(projectId, region, catalogId, dbId, tableId);
+  }
+
+  private String databaseId(Namespace namespace) {
+    return namespace.levels().length == 1 ? namespace.level(0) : null;
+  }
+
+  private DatabaseName databaseName(Namespace namespace) {
+    String db = databaseId(namespace);
+    return db == null ? null : DatabaseName.of(projectId, region, catalogId, db);
+  }
+
+  private Database loadDatabase(Namespace namespace) {
+    return client.getDatabase(databaseName(namespace));
+  }
+
+  private void throwInvalidDbNamespaceError(Namespace namespace) {

Review Comment:
   We typically avoid `throw` methods like this. I think it's a better pattern to have a validation method that does the check and throws if needed, rather than putting the if statement in the caller.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268351544


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()),
+        name.getDatabase());
+  }
+
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getCatalog());
+  }
+
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table getTable(TableName name) {
+    if (name.getTable().isEmpty()) {
+      throw new NoSuchTableException("BigLake API does not allow tables with empty ID");
+    }
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", newName.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table deleteTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.deleteTable(
+                DeleteTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Iterable<Table> listTables(DatabaseName name) {
+    return convertException(
+        () ->
+            stub.listTables(ListTablesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getDatabase());
+  }
+
+  @Override
+  public void close() {
+    stub.close();
+  }
+
+  // Converts BigLake API errors to Iceberg errors.
+  private <T> T convertException(Supplier<T> result, String resourceId) {
+    try {
+      return result.get();
+    } catch (PermissionDeniedException e) {
+      throw new NotAuthorizedException(e, "BigLake API permission denied");
+    } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+      throw new AlreadyExistsException(e, "Namespace already exists: %s", resourceId);

Review Comment:
   Added a comment for clarification.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1266908810


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.ArgumentCaptor;
+
+public class BigLakeTableOperationsTest {
+
+  @TempDir private Path temp;
+
+  private static final String CATALOG_NAME = "iceberg";
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  private static final Schema SCHEMA =
+      new Schema(
+          required(1, "id", Types.IntegerType.get(), "unique ID"),
+          required(2, "data", Types.StringType.get()));
+
+  private BigLakeClient bigLakeClient = mock(BigLakeClient.class);
+  private BigLakeCatalog bigLakeCatalog;
+  private BigLakeTableOperations tableOps;
+
+  @BeforeEach
+  public void before() throws Exception {
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.PROJECT_ID,
+            GCP_PROJECT,
+            GCPProperties.REGION,
+            GCP_REGION,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            temp.toFile().getAbsolutePath(),
+            GCPProperties.BIGLAKE_CATALOG_ID,
+            CATALOG_ID);
+
+    bigLakeCatalog = new BigLakeCatalog();
+    bigLakeCatalog.setConf(new Configuration());
+    bigLakeCatalog.initialize(CATALOG_NAME, properties, bigLakeClient);
+    tableOps = (BigLakeTableOperations) bigLakeCatalog.newTableOps(TABLE_IDENTIFIER);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalog.close();
+  }
+
+  @Test
+  public void testDoCommitShouldUseEtagForUpdateTable() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    Table createdTable = createTestTable();
+
+    Table tableWithEtag = createdTable.toBuilder().setEtag("etag").build();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(tableWithEtag, tableWithEtag);
+
+    org.apache.iceberg.Table loadedTable = bigLakeCatalog.loadTable(TABLE_IDENTIFIER);
+
+    when(bigLakeClient.updateTableParameters(any(), any(), any())).thenReturn(tableWithEtag);
+    loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit();
+
+    ArgumentCaptor<TableName> nameCaptor = ArgumentCaptor.forClass(TableName.class);
+    ArgumentCaptor<String> etagCaptor = ArgumentCaptor.forClass(String.class);
+    verify(bigLakeClient, times(1))
+        .updateTableParameters(nameCaptor.capture(), any(), etagCaptor.capture());
+    assertThat(nameCaptor.getValue()).isEqualTo(TABLE_NAME);
+    assertThat(etagCaptor.getValue()).isEqualTo("etag");
+  }
+
+  @Test
+  public void testDoCommitShouldFailWhenEtagMismatch() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    Table createdTable = createTestTable();
+
+    Table tableWithEtag = createdTable.toBuilder().setEtag("etag").build();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(tableWithEtag, tableWithEtag);
+
+    org.apache.iceberg.Table loadedTable = bigLakeCatalog.loadTable(TABLE_IDENTIFIER);
+
+    when(bigLakeClient.updateTableParameters(any(), any(), any()))
+        .thenThrow(
+            new AbortedException(
+                new RuntimeException("error message etag mismatch"),
+                GrpcStatusCode.of(Code.ABORTED),
+                false));
+
+    assertThatThrownBy(
+            () -> loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit())
+        .isInstanceOf(CommitFailedException.class)
+        .hasMessage("Updating table failed due to conflict updates (etag mismatch)");

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260157793


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";

Review Comment:
   It makes sense. Updated 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260280027


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {

Review Comment:
   Renamed catalogProperties to properties.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260322106


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.

Review Comment:
   Changed it to catalog plugin name and added an example.
   
   Plugin name is like "<plugin-name>" in "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog".



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260367368


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.

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] coufon commented on pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1552514449

   > Just wanted to say that I am really happy to see this! We are looking at using BigLake Metastore and having direct integration in Spark would be a big plus.
   > 
   > One question: what is needed to allow PyIceberg to interface with BigLake Metastore iceberg catalog? I.e., does PyIceberg need a special catalog implementation too? It seems like perhaps it would for writing to the catalog, but if I understand correctly it might be able to read the catalog as if it were a Hive Metastore?
   
   Thanks Matt for your interest. PyIceberg needs an integration here (https://github.com/apache/iceberg/tree/master/python/pyiceberg/catalog) using BigLake Python client (https://cloud.google.com/python/docs/reference/biglake/latest). It has a low priority now. Please send a feature request to biglake-help@google.com to bump its priority up on our end.


-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261837712


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;

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] coufon commented on pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1579707514

   > I think the `BigLakeClient` interface should be replaced with a better way to test the catalog's behavior.
   
   I used a different approach with the real client, and removed the BigLakeClient interface. This new approach uses a mock in-process gRPC server (MockMetastoreServiceImpl.java). Please let me know whether it works.


-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1216924825


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      Preconditions.checkArgument(
+          hiveOptions.containsParameters(METADATA_LOCATION_PROP),
+          "Table %s is not a valid Iceberg table, metadata location not found",
+          tableName());
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  private void createTable(String newMetadataLocation, TableMetadata metadata) {
+    LOG.debug("Creating a new Iceberg table: {}", tableName());
+    client.createTable(tableName, makeNewTable(metadata, newMetadataLocation));
+  }
+
+  /** Update table properties with concurrent update detection using etag. */
+  private void updateTable(
+      String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) {
+    Table table = client.getTable(tableName);
+    String etag = table.getEtag();
+    Preconditions.checkArgument(
+        !etag.isEmpty(),
+        "Etag of legacy table %s is empty, manually update the table by BigLake API or recreate and retry",
+        tableName());
+    HiveTableOptions options = table.getHiveOptions();
+
+    // If `metadataLocationFromMetastore` is different from metadata location of base, it means
+    // someone has updated metadata location in metastore, which is a conflict update.
+    String metadataLocationFromMetastore =
+        options.getParametersOrDefault(METADATA_LOCATION_PROP, "");

Review Comment:
   I think this should use the `getParamtersOrThrow` option.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1216953181


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+
+/** A client interface of Google BigLake service. */
+interface BigLakeClient {

Review Comment:
   This approach also doesn't test any of the code in `BigLakeClientImpl`, like `convertException`. I think this should be removed and replaced by testing against a different `MetastoreServiceClient`.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1192705835


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      Preconditions.checkArgument(
+          hiveOptions.containsParameters(METADATA_LOCATION_PROP),
+          "Table %s is not a valid Iceberg table, metadata location not found",
+          tableName());
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  private void createTable(String newMetadataLocation, TableMetadata metadata) {
+    LOG.debug("Creating a new Iceberg table: {}", tableName());
+    client.createTable(tableName, makeNewTable(metadata, newMetadataLocation));
+  }
+
+  /** Update table properties with concurrent update detection using etag. */
+  private void updateTable(
+      String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) {
+    Table table = client.getTable(tableName);
+    String etag = table.getEtag();
+    Preconditions.checkArgument(
+        !etag.isEmpty(),
+        "Etag of legacy table %s is empty, manually update the table by BigLake API or recreate and retry",
+        tableName());
+    HiveTableOptions options = table.getHiveOptions();
+
+    // If `metadataLocationFromMetastore` is different from metadata location of base, it means
+    // someone has updated metadata location in metastore, which is a conflict update.
+    String metadataLocationFromMetastore =
+        options.getParametersOrDefault(METADATA_LOCATION_PROP, "");
+    if (!metadataLocationFromMetastore.isEmpty()
+        && !metadataLocationFromMetastore.equals(oldMetadataLocation)) {
+      throw new CommitFailedException(
+          "Base metadata location '%s' is not same as the current table metadata location '%s' for"
+              + " %s.%s",
+          oldMetadataLocation,
+          metadataLocationFromMetastore,
+          tableName.getDatabase(),
+          tableName.getTable());
+    }
+
+    try {
+      // Updating a BLMS table with etag. The BLMS server transactionally (1) checks that the etag
+      // of a table on server is the same as the etag provided by the client, and (2) updates the
+      // table (and its etag). The server returns an error containing message "etag mismatch", if
+      // the etag on server has changed.
+      client.updateTableParameters(
+          tableName, buildTableParameters(newMetadataLocation, metadata), etag);
+    } catch (AbortedException e) {
+      if (e.getMessage().toLowerCase().contains("etag mismatch")) {
+        throw new CommitFailedException(
+            "Updating table failed due to conflict updates (etag mismatch)");
+      }
+    }
+  }
+
+  private Table makeNewTable(TableMetadata metadata, String metadataFileLocation) {
+    Table.Builder builder = Table.newBuilder().setType(Table.Type.HIVE);

Review Comment:
   This will be the next step of Iceberg integration in the BigLake API. The current API follows how HMS stores Iceberg, it has the benefit that a HMS proxy can talk to our API, to integrate with any engines supporting HMS. We will look into adding an Iceberg type and exploring how Iceberg Rest client can seamlessly access BigLake API.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1186929021


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      Preconditions.checkArgument(
+          hiveOptions.containsParameters(METADATA_LOCATION_PROP),
+          "Table %s is not a valid Iceberg table, metadata location not found",
+          tableName());
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  private void createTable(String newMetadataLocation, TableMetadata metadata) {
+    LOG.debug("Creating a new Iceberg table: {}", tableName());
+    client.createTable(tableName, makeNewTable(metadata, newMetadataLocation));
+  }
+
+  /** Update table properties with concurrent update detection using etag. */
+  private void updateTable(
+      String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) {
+    Table table = client.getTable(tableName);
+    String etag = table.getEtag();
+    Preconditions.checkArgument(
+        !etag.isEmpty(),
+        "Etag of legacy table %s is empty, manually update the table by BigLake API or recreate and retry",
+        tableName());
+    HiveTableOptions options = table.getHiveOptions();
+
+    // If `metadataLocationFromMetastore` is different from metadata location of base, it means
+    // someone has updated metadata location in metastore, which is a conflict update.
+    String metadataLocationFromMetastore =
+        options.getParametersOrDefault(METADATA_LOCATION_PROP, "");
+    if (!metadataLocationFromMetastore.isEmpty()
+        && !metadataLocationFromMetastore.equals(oldMetadataLocation)) {
+      throw new CommitFailedException(
+          "Base metadata location '%s' is not same as the current table metadata location '%s' for"
+              + " %s.%s",
+          oldMetadataLocation,
+          metadataLocationFromMetastore,
+          tableName.getDatabase(),
+          tableName.getTable());
+    }
+
+    try {
+      // Updating a BLMS table with etag. The BLMS server transactionally (1) checks that the etag
+      // of a table on server is the same as the etag provided by the client, and (2) updates the
+      // table (and its etag). The server returns an error containing message "etag mismatch", if
+      // the etag on server has changed.
+      client.updateTableParameters(
+          tableName, buildTableParameters(newMetadataLocation, metadata), etag);
+    } catch (AbortedException e) {
+      if (e.getMessage().toLowerCase().contains("etag mismatch")) {
+        throw new CommitFailedException(
+            "Updating table failed due to conflict updates (etag mismatch)");
+      }

Review Comment:
   This needs to handle the else case. Otherwise, an abort would appear to succeed and could result in data loss.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260316136


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260156760


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.

Review Comment:
   I made the changes in this PR.



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

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260355786


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();
+    } else if (namespace.levels().length == 1) {
+      return getMetadata(getDatabase(namespace));
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogPulginName;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return catalogProperties == null ? ImmutableMap.of() : catalogProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  private String getDatabaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            catalogProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier getTableIdentifier(Table table) {

Review Comment:
   Done. Fixed all of them.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1272527672


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()),
+        name.getDatabase());
+  }
+
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getCatalog());
+  }
+
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table getTable(TableName name) {
+    if (name.getTable().isEmpty()) {
+      throw new NoSuchTableException("BigLake API does not allow tables with empty ID");
+    }
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", newName.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table deleteTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.deleteTable(
+                DeleteTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Iterable<Table> listTables(DatabaseName name) {
+    return convertException(
+        () ->
+            stub.listTables(ListTablesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getDatabase());
+  }
+
+  @Override
+  public void close() {
+    stub.close();
+  }
+
+  // Converts BigLake API errors to Iceberg errors.
+  private <T> T convertException(Supplier<T> result, String resourceId) {
+    try {
+      return result.get();
+    } catch (PermissionDeniedException e) {
+      throw new NotAuthorizedException(e, "BigLake API permission denied");

Review Comment:
   I think it's valuable if you can tell the caller what permission was missing. But a general permission denied exception doesn't need the source, since it's clear this is coming from BigLake (it's in the stack trace).



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1262403576


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.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.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient {
+
+  private final String projectId;
+  private final String region;

Review Comment:
   `projectId` and `region` don't seem to be used within this file



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO io;
+  // The catalog name.
+  private final String name;

Review Comment:
   not sure if we need the comment above, maybe just rename the variable to `catalogName`?



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {

Review Comment:
   pretty much all `throws Exception` on test methods can be removed. Same applies for `BigLakeTableOperationsTest`



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))

Review Comment:
   ```suggestion
       assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of("n0", "n1")))
   ```



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();

Review Comment:
   should those maybe all have a null check? Because if test initialization fails, then these might end up being null



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {

Review Comment:
   `invalid` is a bit misleading here. Is it really invalid to list an unknown namespace?



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))
+        .isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {}), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {"db", "tbl"}), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);

Review Comment:
   this one and the one above should have a `hasMessage(..)` check



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))
+        .isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {}), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {"db", "tbl"}), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+  }
+
+  @Test
+  public void testSetPropertiesShouldSucceedForDatabase() throws Exception {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.setProperties(
+                Namespace.of(new String[] {"db"}),
+                ImmutableMap.of("key2", "value222", "key3", "value3")))
+        .isTrue();
+    verify(mockBigLakeClient, times(1))
+        .updateDatabaseParameters(
+            DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db"),
+            ImmutableMap.of("key1", "value1", "key2", "value222", "key3", "value3"));
+  }
+
+  @Test
+  public void testRemovePropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.removeProperties(
+                    Namespace.of(new String[] {}), ImmutableSet.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.removeProperties(
+                    Namespace.of(new String[] {"db", "tbl"}), ImmutableSet.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);

Review Comment:
   `.hasMessage()` here and in all other places



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");

Review Comment:
   and then the `warehouseLocation` variable isn't required



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              projectId,
+              region);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      bigLakeClient.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return metadata(loadDatabase(namespace));
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return bigLakeProperties == null ? ImmutableMap.of() : bigLakeProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    closeableGroup.close();
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            bigLakeProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier tableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace namespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName tableName(String dbId, String tableId) {
+    return TableName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId, tableId);
+  }
+
+  private String databaseId(Namespace namespace) {
+    if (namespace.levels().length != 1) {
+      throw new NoSuchNamespaceException(
+          "BigLake database namespace must use format <catalog>.<database>, invalid namespace: %s",
+          namespace);
+    }
+
+    return namespace.level(0);
+  }
+
+  private DatabaseName databaseName(Namespace namespace) {
+    return DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, databaseId(namespace));
+  }
+
+  private Database loadDatabase(Namespace namespace) {
+    return bigLakeClient.getDatabase(databaseName(namespace));
+  }
+
+  private static Map<String, String> metadata(Database db) {
+    HiveDatabaseOptions options = db.getHiveOptions();
+    return new ImmutableMap.Builder<String, String>()
+        .putAll(options.getParameters())

Review Comment:
   seems to be deprecated, should this use `getParametersMap()`?



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO io;
+  // The catalog name.
+  private final String name;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO io, String name, TableName tableName) {
+    this.client = client;
+    this.io = io;
+    this.name = name;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new NoSuchIcebergTableException(
+            "Invalid Iceberg table %s: missing metadata location", tableName());
+      }
+
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (AlreadyExistsException | CommitFailedException | CommitStateUnknownException e) {

Review Comment:
   should the commit status be set to `CommitStatus.UNKNOWN` before throwing `CommitStateUnknownException`?



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))

Review Comment:
   same for all the other places that follow a similar pattern



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              projectId,
+              region);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      bigLakeClient.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return metadata(loadDatabase(namespace));
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return bigLakeProperties == null ? ImmutableMap.of() : bigLakeProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    closeableGroup.close();

Review Comment:
   I think this needs a null check, as it's possible that `closeableGroup` ends up being `null` when some initialization part fails



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {

Review Comment:
   I'm not sure the `_asExpected` is helpful here. IMO the expected is always implicit, so this can be removed



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))
+        .isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {}), ImmutableMap.of()))

Review Comment:
   ```suggestion
                       Namespace.empty(), ImmutableMap.of()))
   ```



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);

Review Comment:
   I think this can be just switched to using `temp.toAbsolutePath().toString()` here and further below where the `warehouseLocation` var is being used. Then `warehouseLocation` can be removed



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;

Review Comment:
   seems that `mockMetastoreService` and `channelProvider` can be converted to a local variable



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {

Review Comment:
   I think a better name would be `renameTableToDifferentDatabaseShouldFail()`. The `test` prefix doesn't add any value and comes from ancient JUnit versions due to Java limitations. I'm aware that we're using this prefix throughout the codebase, but I don't think we should be adding it for new test code



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");

Review Comment:
   ```suggestion
           .isEqualTo(temp.toAbsolutePath() + "/db.db/table");
   ```



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {

Review Comment:
   the name here is misleading. First of all it indicates that something should fail and that an unknown namespace is invalid



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))
+        .isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {}), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {"db", "tbl"}), ImmutableMap.of()))

Review Comment:
   ```suggestion
                       Namespace.of("db", "tbl"), ImmutableMap.of()))
   ```



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));

Review Comment:
   ```suggestion
       bigLakeCatalogUsingMockClient.dropNamespace(Namespace.empty());
   ```



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))
+        .isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {}), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {"db", "tbl"}), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+  }
+
+  @Test
+  public void testSetPropertiesShouldSucceedForDatabase() throws Exception {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.setProperties(
+                Namespace.of(new String[] {"db"}),
+                ImmutableMap.of("key2", "value222", "key3", "value3")))
+        .isTrue();
+    verify(mockBigLakeClient, times(1))
+        .updateDatabaseParameters(
+            DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db"),
+            ImmutableMap.of("key1", "value1", "key2", "value222", "key3", "value3"));
+  }
+
+  @Test
+  public void testRemovePropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.removeProperties(
+                    Namespace.of(new String[] {}), ImmutableSet.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);

Review Comment:
   I've noticed that this one ends up showing: `BigLake database namespace must use format <catalog>.<database>, invalid namespace: ` which is a bit misleading. Maybe for an empty namespace the error msg should clearly indicate `...Invalid namespace: empty` or something along those lines



##########
build.gradle:
##########
@@ -608,18 +608,42 @@ project(':iceberg-gcp') {
 
     implementation platform('com.google.cloud:libraries-bom')
     implementation 'com.google.cloud:google-cloud-storage'
-
-    testImplementation 'com.google.cloud:google-cloud-nio'
+    implementation 'com.google.cloud:google-cloud-biglake'
+    compileOnly('org.apache.hadoop:hadoop-common') {
+      exclude group: 'org.apache.avro', module: 'avro'
+      exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+      exclude group: 'javax.servlet', module: 'servlet-api'
+      exclude group: 'com.google.code.gson', module: 'gson'
+    }
+    compileOnly("org.apache.hive:hive-metastore") {

Review Comment:
   is this dependency required? the project was compiling fine for me without it



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.ArgumentCaptor;
+
+public class BigLakeTableOperationsTest {
+
+  @TempDir private Path temp;
+
+  private static final String CATALOG_NAME = "iceberg";
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier SPARK_TABLE_ID = TableIdentifier.of(DB_ID, TABLE_ID);

Review Comment:
   why `SPARK_TABLE_ID` and not just `IDENTIFIER` or something else, as it's not specific to Spark



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              projectId,
+              region);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();

Review Comment:
   I think this should init `ops` and read current tablemetadata only if `purge` is set to `true`



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))
+        .isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {}), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {"db", "tbl"}), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+  }
+
+  @Test
+  public void testSetPropertiesShouldSucceedForDatabase() throws Exception {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.setProperties(
+                Namespace.of(new String[] {"db"}),
+                ImmutableMap.of("key2", "value222", "key3", "value3")))
+        .isTrue();
+    verify(mockBigLakeClient, times(1))
+        .updateDatabaseParameters(
+            DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db"),
+            ImmutableMap.of("key1", "value1", "key2", "value222", "key3", "value3"));
+  }
+
+  @Test
+  public void testRemovePropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.removeProperties(
+                    Namespace.of(new String[] {}), ImmutableSet.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.removeProperties(
+                    Namespace.of(new String[] {"db", "tbl"}), ImmutableSet.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+  }
+
+  @Test
+  public void testRemovePropertiesShouldSucceedForDatabase() throws Exception {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.removeProperties(
+                Namespace.of(new String[] {"db"}), ImmutableSet.of("key1", "key3")))
+        .isTrue();
+    verify(mockBigLakeClient, times(1))
+        .updateDatabaseParameters(
+            DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db"),
+            ImmutableMap.of("key2", "value2"));
+  }
+
+  @Test
+  public void testLoadNamespaceMetadataAsExpectedForCatalogs() throws Exception {

Review Comment:
   maybe `emptyNamespaceLoadsEmptyNamespaceMetadata` or something similar. The `AsExpected` is always implicit, because as a reader of the test, I'm always expecting whathever the test does



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))
+        .isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {}), ImmutableMap.of()))

Review Comment:
   there are also a bunch of other places that do unnecessary array initialization that should be updated



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              projectId,
+              region);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      bigLakeClient.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return metadata(loadDatabase(namespace));
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return bigLakeProperties == null ? ImmutableMap.of() : bigLakeProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    closeableGroup.close();
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            bigLakeProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier tableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace namespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName tableName(String dbId, String tableId) {
+    return TableName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId, tableId);
+  }
+
+  private String databaseId(Namespace namespace) {
+    if (namespace.levels().length != 1) {
+      throw new NoSuchNamespaceException(
+          "BigLake database namespace must use format <catalog>.<database>, invalid namespace: %s",
+          namespace);
+    }
+
+    return namespace.level(0);
+  }
+
+  private DatabaseName databaseName(Namespace namespace) {
+    return DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, databaseId(namespace));
+  }
+
+  private Database loadDatabase(Namespace namespace) {
+    return bigLakeClient.getDatabase(databaseName(namespace));
+  }
+
+  private static Map<String, String> metadata(Database db) {
+    HiveDatabaseOptions options = db.getHiveOptions();
+    return new ImmutableMap.Builder<String, String>()
+        .putAll(options.getParameters())
+        .put("location", options.getLocationUri())

Review Comment:
   is the `location` used internally?



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264603919


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))
+        .isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {}), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {"db", "tbl"}), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+  }
+
+  @Test
+  public void testSetPropertiesShouldSucceedForDatabase() throws Exception {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.setProperties(
+                Namespace.of(new String[] {"db"}),
+                ImmutableMap.of("key2", "value222", "key3", "value3")))
+        .isTrue();
+    verify(mockBigLakeClient, times(1))
+        .updateDatabaseParameters(
+            DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db"),
+            ImmutableMap.of("key1", "value1", "key2", "value222", "key3", "value3"));
+  }
+
+  @Test
+  public void testRemovePropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.removeProperties(
+                    Namespace.of(new String[] {}), ImmutableSet.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.removeProperties(
+                    Namespace.of(new String[] {"db", "tbl"}), ImmutableSet.of()))
+        .isInstanceOf(NoSuchNamespaceException.class);
+  }
+
+  @Test
+  public void testRemovePropertiesShouldSucceedForDatabase() throws Exception {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.removeProperties(
+                Namespace.of(new String[] {"db"}), ImmutableSet.of("key1", "key3")))
+        .isTrue();
+    verify(mockBigLakeClient, times(1))
+        .updateDatabaseParameters(
+            DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db"),
+            ImmutableMap.of("key2", "value2"));
+  }
+
+  @Test
+  public void testLoadNamespaceMetadataAsExpectedForCatalogs() throws Exception {

Review Comment:
   Changed to testEmptyNamespaceLoadsCatalogMetadata since it calls the server to get catalog resource metadata.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261422667


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.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.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID), "GCP region must be specified");

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] nastra commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261357299


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();

Review Comment:
   I'm not sure this will work because `@Rule` is a JUnit4 thing but these tests are running with JUnit5. Also could you please elaborate why this rule is needed?



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.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.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID), "GCP region must be specified");

Review Comment:
   ```suggestion
           properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
   ```



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.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.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {

Review Comment:
   ```suggestion
     public void initialize(String name, Map<String, String> properties) {
   ```



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.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.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              projectId,
+              region);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.catalogPulginName = inputName;
+    this.properties = ImmutableMap.copyOf(properties);
+    this.projectId = projectId;
+    this.region = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.client = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = this.properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // Return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, io, tableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client, io, tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(databaseName(namespace)))
+        .map(BigLakeCatalog::tableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): %s", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {

Review Comment:
   `namespace.isEmpty()`



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.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.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              projectId,
+              region);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.catalogPulginName = inputName;
+    this.properties = ImmutableMap.copyOf(properties);
+    this.projectId = projectId;
+    this.region = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.client = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = this.properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // Return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, io, tableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client, io, tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(databaseName(namespace)))
+        .map(BigLakeCatalog::tableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {

Review Comment:
   `!namespace.isEmpty()`



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;

Review Comment:
   Iceberg is currently migrating from JUnit4 to JUnit5, so it would be great to not use any JUnit4 style assertions and rather switch directly to AssertJ: https://iceberg.apache.org/contribute/#assertj



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    mockBigLakeClient = mock(BigLakeClient.class);
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() {
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertEquals(
+        "db_folder/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertEquals(
+        warehouseLocation + "/db.db/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    Exception exception =

Review Comment:
   please use AssertJ-style assertions for this. This would be something like: `assertThatThrownBy(() -> client.renameTable(...)).isInstanceOf(IllegalArgumentException.class).hasMessage(...)`.



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;

Review Comment:
   I believe this can be made `private`



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;

Review Comment:
   same as above regarding JUnit4-style assertions



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    mockBigLakeClient = mock(BigLakeClient.class);
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() {
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertEquals(
+        "db_folder/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertEquals(
+        warehouseLocation + "/db.db/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    Exception exception =
+        assertThrows(
+            IllegalArgumentException.class,
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")));
+    assertEquals(
+        "Cannot rename table db0.t1 to db1.t2: database must match", exception.getMessage());
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {

Review Comment:
   I believe test methods shouldn't contain `_` and rather use camelCase



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import java.io.File;
+import java.io.IOException;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;

Review Comment:
   same as above, new tests being added should be JUnit5: https://iceberg.apache.org/contribute/#junit4--junit5



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1266091509


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkNotNull(initClient, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    return new BigLakeTableOperations(client, io, name(), tableName(db, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(db) : locationUri, identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else if (namespace.levels().length == 1) {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    } else {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(db, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    if (fromDb == null) {
+      throwInvalidDbNamespaceError(from.namespace());
+    }
+
+    String toDb = databaseId(to.namespace());
+    if (toDb == null) {
+      throwInvalidDbNamespaceError(to.namespace());
+    }
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> props) {
+    DatabaseName dbName = databaseName(namespace);
+    if (dbName == null) {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(dbName, optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> props) {
+    DatabaseName dbName = databaseName(namespace);
+    if (dbName == null) {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(dbName, optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return loadDatabase(namespace).getHiveOptions().getParametersMap();
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return properties == null ? ImmutableMap.of() : properties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closeableGroup != null) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264759044


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty, returns empty to unblock
+      // deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      bigLakeClient.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return metadata(loadDatabase(namespace));
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return bigLakeProperties == null ? ImmutableMap.of() : bigLakeProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closeableGroup != null) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            bigLakeProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier tableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace namespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName tableName(String dbId, String tableId) {
+    return TableName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId, tableId);
+  }
+
+  private String databaseId(Namespace namespace) {
+    if (namespace.levels().length != 1) {
+      throw new NoSuchNamespaceException(

Review Comment:
   Done.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty, returns empty to unblock
+      // deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      bigLakeClient.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return metadata(loadDatabase(namespace));
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return bigLakeProperties == null ? ImmutableMap.of() : bigLakeProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closeableGroup != null) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            bigLakeProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier tableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace namespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName tableName(String dbId, String tableId) {
+    return TableName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId, tableId);
+  }
+
+  private String databaseId(Namespace namespace) {
+    if (namespace.levels().length != 1) {
+      throw new NoSuchNamespaceException(
+          namespace.isEmpty()
+              ? "Invalid BigLake database namespace: empty"
+              : String.format(
+                  "BigLake database namespace must use format <catalog>.<database>, invalid namespace: %s",
+                  namespace));
+    }
+
+    return namespace.level(0);
+  }
+
+  private DatabaseName databaseName(Namespace namespace) {
+    return DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, databaseId(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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264759136


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty, returns empty to unblock
+      // deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261872286


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;

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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268253201


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()),
+        name.getDatabase());
+  }
+
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getCatalog());
+  }
+
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table getTable(TableName name) {
+    if (name.getTable().isEmpty()) {
+      throw new NoSuchTableException("BigLake API does not allow tables with empty ID");
+    }
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", newName.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table deleteTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.deleteTable(
+                DeleteTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Iterable<Table> listTables(DatabaseName name) {
+    return convertException(
+        () ->
+            stub.listTables(ListTablesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getDatabase());
+  }
+
+  @Override
+  public void close() {
+    stub.close();
+  }
+
+  // Converts BigLake API errors to Iceberg errors.
+  private <T> T convertException(Supplier<T> result, String resourceId) {
+    try {
+      return result.get();
+    } catch (PermissionDeniedException e) {
+      throw new NotAuthorizedException(e, "BigLake API permission denied");

Review Comment:
   What is the value of "BigLake" in this exception message? We typically try to standardize.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268254014


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()),
+        name.getDatabase());
+  }
+
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getCatalog());
+  }
+
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table getTable(TableName name) {
+    if (name.getTable().isEmpty()) {
+      throw new NoSuchTableException("BigLake API does not allow tables with empty ID");
+    }
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {

Review Comment:
   Why use `convertException` when this catches the same exceptions that `convertException` handles?



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268320271


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,404 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    return new BigLakeTableOperations(client, io, name(), tableName(db, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(db) : locationUri, identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else if (namespace.levels().length == 1) {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    } else {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(db, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    if (fromDb == null) {
+      throwInvalidDbNamespaceError(from.namespace());
+    }
+
+    String toDb = databaseId(to.namespace());
+    if (toDb == null) {
+      throwInvalidDbNamespaceError(to.namespace());
+    }
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> props) {
+    DatabaseName dbName = databaseName(namespace);
+    if (dbName == null) {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(dbName, optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> props) {
+    DatabaseName dbName = databaseName(namespace);
+    if (dbName == null) {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(dbName, optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return loadDatabase(namespace).getHiveOptions().getParametersMap();
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return properties == null ? ImmutableMap.of() : properties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closeableGroup != null) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    return String.format("%s/%s.db", warehouseLocation, dbId);
+  }
+
+  private static TableIdentifier tableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace namespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName tableName(String dbId, String tableId) {
+    return TableName.of(projectId, region, catalogId, dbId, tableId);
+  }
+
+  private String databaseId(Namespace namespace) {
+    return namespace.levels().length == 1 ? namespace.level(0) : null;
+  }
+
+  private DatabaseName databaseName(Namespace namespace) {
+    String db = databaseId(namespace);
+    return db == null ? null : DatabaseName.of(projectId, region, catalogId, db);
+  }
+
+  private Database loadDatabase(Namespace namespace) {
+    return client.getDatabase(databaseName(namespace));
+  }
+
+  private void throwInvalidDbNamespaceError(Namespace namespace) {
+    throw new NoSuchNamespaceException(
+        "Invalid BigLake database namespace: %s", namespace.isEmpty() ? "empty" : 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1266949036


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    return new BigLakeTableOperations(client, io, name(), tableName(db, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(db) : locationUri, identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else if (namespace.levels().length == 1) {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    } else {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(db, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    if (fromDb == null) {
+      throwInvalidDbNamespaceError(from.namespace());
+    }
+
+    String toDb = databaseId(to.namespace());
+    if (toDb == null) {
+      throwInvalidDbNamespaceError(to.namespace());
+    }
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> props) {
+    DatabaseName dbName = databaseName(namespace);
+    if (dbName == null) {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(dbName, optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> props) {
+    DatabaseName dbName = databaseName(namespace);
+    if (dbName == null) {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(dbName, optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return loadDatabase(namespace).getHiveOptions().getParametersMap();
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return properties == null ? ImmutableMap.of() : properties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closeableGroup != null) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkArgument(warehouseLocation != null, "Data warehouse location is not set");

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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253561536


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,

Review Comment:
   No need for the `prop` prefix.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {

Review Comment:
   How about just `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] rdblue commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253557372


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);

Review Comment:
   There should be whitespace before this line.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253616870


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.

Review Comment:
   Avoid using personal pronouns like "I" or "We" in comments and documentation. Using first person doesn't help make the comment more clear, only longer. Omitting the personal pronoun usually makes the comment more direct and useful.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253604158


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));

Review Comment:
   I think this should be `false`. The namespace can't exist because the name is wrong, so this should return false, indicating that no action was taken and the namespace does not exist.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253603137


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();
+    } else if (namespace.levels().length == 1) {
+      return getMetadata(getDatabase(namespace));

Review Comment:
   This returns a mutable map. Can you use an ImmutableMap instead?



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();

Review Comment:
   Also mutable 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] rdblue commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253610509


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");

Review Comment:
   Can you add the mismatched database names?
   
   Also, Iceberg error messages try to be more clear, with the problem up front and clarifying information after. I think this should be "Cannot rename table %s to %s: database must match"



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335234322


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        String dbId = databaseId(namespace);
+        validateDatabaseId(dbId, namespace);

Review Comment:
   This isn't needed.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335232431


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,

Review Comment:
   I can see that the intent here is for `databaseLocation` to be the default if the location is not explicitly set in metadata, but this naming is confusing. I would expect `databaseLocation(dbId)` to always produce the correct location (whether stored in the metastore or not) because that's what the name implies.
   
   I think this should either rename `databaseLocation` to something more `defaultWarehouseLocation(Namespace)` or `newLocation(databaseId)`. If you use this naming, then move `loadDatabase(dbId).getHiveOptions().getLocationUri()` into that 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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344488229


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        String dbId = databaseId(namespace);
+        validateDatabaseId(dbId, 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1192716941


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeTableOperationsTest {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier SPARK_TABLE_ID = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  @Mock private BigLakeClient bigLakeClient;
+
+  private BigLakeCatalog bigLakeCatalog;
+  private String warehouseLocation;
+  private BigLakeTableOperations tableOps;
+
+  @Before
+  public void before() throws Exception {
+    this.bigLakeCatalog = new BigLakeCatalog();
+    this.warehouseLocation = tempFolder.newFolder("hive-warehouse").toString();
+
+    bigLakeCatalog.setConf(new Configuration());
+    bigLakeCatalog.initialize(
+        CATALOG_ID,
+        /* properties= */ ImmutableMap.of(
+            BigLakeCatalog.PROPERTIES_KEY_GCP_PROJECT,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation),
+        GCP_PROJECT,
+        GCP_REGION,
+        bigLakeClient);
+    this.tableOps = (BigLakeTableOperations) bigLakeCatalog.newTableOps(SPARK_TABLE_ID);
+  }
+
+  @Test
+  public void testDoFresh_fetchLatestMetadataFromBigLake() throws Exception {
+    Table createdTable = createTestTable();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(createdTable);
+
+    tableOps.refresh();
+    assertEquals(
+        createdTable
+            .getHiveOptions()
+            .getParametersOrDefault(BigLakeTestUtils.METADATA_LOCATION_PROP, ""),
+        tableOps.currentMetadataLocation());
+
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    // Refresh fails when table is not found but metadata already presents.
+    assertThrows(NoSuchTableException.class, () -> tableOps.refresh());
+  }
+
+  @Test
+  public void testDoFresh_failForNonIcebergTable() throws Exception {

Review Comment:
   Thanks. I updated the code to throw NoSuchIcebergTableException and updated the test as well.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1192699301


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+
+/** A client interface of Google BigLake service. */
+interface BigLakeClient {

Review Comment:
   The interface is for creating a fake client implementation. In the previous commit, I use mocked clients so this interface was not used. But now I switched to CatalogTests as you suggested, mocking client per test case doesn't work any more, so I added a FakeBigLakeClient implementing this interface for tests.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1179791110


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "gcp_location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "blms_catalog";
+
+  public static final String HIVE_METASTORE_WAREHOUSE_DIR = "hive.metastore.warehouse.dir";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileSystem fs;
+  private FileIO fileIO;
+  private Configuration conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    if (!properties.containsKey(PROPERTIES_KEY_GCP_PROJECT)) {

Review Comment:
   You should probably switch this to use `ValidationException.check(...)` or `Preconditions.check`.  The latter approach is more consistent with other initializations.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";

Review Comment:
   These properties appear to be duplicates of values in `GCPProperties`.  Please use that class instead for defining and accessing properties.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "gcp_location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "blms_catalog";
+
+  public static final String HIVE_METASTORE_WAREHOUSE_DIR = "hive.metastore.warehouse.dir";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileSystem fs;
+  private FileIO fileIO;
+  private Configuration conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    if (!properties.containsKey(PROPERTIES_KEY_GCP_PROJECT)) {
+      throw new ValidationException("GCP project must be specified");
+    }
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+    BigLakeClient newClient;
+    try {
+      newClient =
+          new BigLakeClientImpl(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    if (this.conf == null) {
+      LOG.warn("No Hadoop Configuration was set, using the default environment Configuration");
+      this.conf = new Configuration();
+    }
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) {
+      this.conf.set(
+          HIVE_METASTORE_WAREHOUSE_DIR,
+          LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION)));
+    }
+
+    this.fs =
+        Util.getFs(
+            new Path(
+                LocationUtil.stripTrailingSlash(
+                    properties.get(CatalogProperties.WAREHOUSE_LOCATION))),
+            conf);
+
+    String fileIOImpl =
+        properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.hadoop.HadoopFileIO");
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        conf,
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    client.deleteTable(
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    if (!fromDbId.equals(toDbId)) {
+      throw new ValidationException("New table name must be in the same database");
+    }
+
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {

Review Comment:
   I'm not sure this will map correctly to the spark catalog namespacing.  What is the namespacing for the catalog?
   
   Is it `<spark-catalog>.<biglake-catalog>.<database/schema>.<table>`?



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final Configuration conf;

Review Comment:
   We should avoid direct dependencies on Hadoop.  I don't why this is necessary as it does not appear to be used anywhere.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+
+/** A client interface of Google BigLake service. */
+public interface BigLakeClient {

Review Comment:
   This doesn't need to be public



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {

Review Comment:
   We should use `Configurable<Object>` and avoid direct dependencies on Hadoop.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "gcp_location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "blms_catalog";
+
+  public static final String HIVE_METASTORE_WAREHOUSE_DIR = "hive.metastore.warehouse.dir";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileSystem fs;
+  private FileIO fileIO;
+  private Configuration conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    if (!properties.containsKey(PROPERTIES_KEY_GCP_PROJECT)) {
+      throw new ValidationException("GCP project must be specified");
+    }
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+    BigLakeClient newClient;
+    try {
+      newClient =
+          new BigLakeClientImpl(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    if (this.conf == null) {
+      LOG.warn("No Hadoop Configuration was set, using the default environment Configuration");
+      this.conf = new Configuration();
+    }
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) {
+      this.conf.set(
+          HIVE_METASTORE_WAREHOUSE_DIR,
+          LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION)));
+    }
+
+    this.fs =
+        Util.getFs(
+            new Path(
+                LocationUtil.stripTrailingSlash(
+                    properties.get(CatalogProperties.WAREHOUSE_LOCATION))),
+            conf);
+
+    String fileIOImpl =
+        properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.hadoop.HadoopFileIO");

Review Comment:
   `ResolvingFileIO` should be used as the default.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final Configuration conf;
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(
+      Configuration conf, BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.conf = conf;
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new ValidationException(
+            "Table %s is not a valid Iceberg table, metadata location not found", tableName());
+      }
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  private void createTable(String newMetadataLocation, TableMetadata metadata) {
+    LOG.debug("Creating a new Iceberg table: {}", tableName());
+    client.createTable(tableName, makeNewTable(metadata, newMetadataLocation));
+  }
+
+  /** Update table properties with concurrent update detection using etag. */
+  private void updateTable(
+      String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) {
+    Table table = client.getTable(tableName);
+    String etag = table.getEtag();
+    if (etag.isEmpty()) {
+      throw new ValidationException(
+          "Etag of legacy table %s is empty, manually update the table by BigLake API or"
+              + " recreate and retry",
+          tableName());
+    }
+    HiveTableOptions options = table.getHiveOptions();
+
+    // If `metadataLocationFromMetastore` is different from metadata location of base, it means
+    // someone has updated metadata location in metastore, which is a conflict update.
+    String metadataLocationFromMetastore =
+        options.getParametersOrDefault(METADATA_LOCATION_PROP, "");
+    if (!metadataLocationFromMetastore.isEmpty()
+        && !metadataLocationFromMetastore.equals(oldMetadataLocation)) {
+      throw new CommitFailedException(
+          "Base metadata location '%s' is not same as the current table metadata location '%s' for"
+              + " %s.%s",
+          oldMetadataLocation,
+          metadataLocationFromMetastore,
+          tableName.getDatabase(),
+          tableName.getTable());
+    }
+
+    try {
+      client.updateTableParameters(
+          tableName, buildTableParameters(newMetadataLocation, metadata), etag);
+    } catch (AbortedException e) {
+      if (e.getMessage().toLowerCase().contains("etag mismatch")) {
+        throw new CommitFailedException(
+            "Updating table failed due to conflict updates (etag mismatch)");
+      }
+    }
+  }
+
+  private Table makeNewTable(TableMetadata metadata, String metadataFileLocation) {
+    Table.Builder builder = Table.newBuilder().setType(Table.Type.HIVE);
+    builder
+        .getHiveOptionsBuilder()

Review Comment:
   Is this necessary?  These options don't apply to Iceberg tables.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClientImpl.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client implementation of Google BigLake service. */
+public final class BigLakeClientImpl implements BigLakeClient {

Review Comment:
   This should be package 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1180597983


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "gcp_location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "blms_catalog";
+
+  public static final String HIVE_METASTORE_WAREHOUSE_DIR = "hive.metastore.warehouse.dir";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileSystem fs;
+  private FileIO fileIO;
+  private Configuration conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    if (!properties.containsKey(PROPERTIES_KEY_GCP_PROJECT)) {

Review Comment:
   Thanks. I changed all checks to Preconditions.checkArguments.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1180611269


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";

Review Comment:
   It makes sense. I added a TODO to use GCPProperties in a following PR. I changed the config name to follow the existing style (e.g., biglake.project-id) in this PR. The issue of simply using GCPProperties in this PR is that "gcs.project-id" is too GCS specific, but the class field name is "projectId". I hope to change it to gcsProjectId, and add a bigLakeProjectId. It touches other classes so I prefer to separate it out.
   
   I actually feel "gcs.project-id" is not necessary: it would be great if customers can use buckets from any project (and different projects) in the same catalog handler. Most GCP customers today use a GCS connector (https://cloud.google.com/dataproc/docs/concepts/connectors/cloud-storage). It is an adapter between HDFS and GCS. A URL like "gs://bucket/folder" will be handled by this connector, project ID is detected from the bucket. It is open-source and pre-installed in Dataproc. This makes me believe specifying a project ID for GCS is not needed.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1186927171


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClientImpl.java:
##########
@@ -0,0 +1,287 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client implementation of Google BigLake service. */
+final class BigLakeClientImpl implements BigLakeClient {
+
+  private final String projectId;
+  private final String location;
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   * @param projectId GCP project ID
+   * @param location GCP region supported by BigLake, e.g., "us"
+   */
+  BigLakeClientImpl(String biglakeEndpoint, String projectId, String location) throws IOException {
+    this.projectId = projectId;
+    this.location = location;
+    this.stub =
+        MetastoreServiceClient.create(
+            MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  @Override
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()));
+  }
+
+  @Override
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Catalog %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Catalog %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()));
+  }
+
+  @Override
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Database %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Database %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll());
+  }
+
+  @Override
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Database %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () ->
+            stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build()));
+  }
+
+  @Override
+  public Table getTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Table deleteTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.deleteTable(
+                DeleteTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table %s not found or permission denied", name.toString());
+          }
+        });
+  }
+
+  @Override
+  public Iterable<Table> listTables(DatabaseName name) {
+    return convertException(
+        () ->
+            stub.listTables(ListTablesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll());
+  }
+
+  // Converts BigLake API errors to Iceberg errors.
+  private <T> T convertException(Supplier<T> result) {
+    try {
+      return result.get();
+    } catch (PermissionDeniedException e) {
+      throw new NotAuthorizedException(e, "Not authorized to call BigLake API");

Review Comment:
   I think this is incorrect. The person may be authorized to call the API, but not authorized for the attempted action.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1186928721


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      Preconditions.checkArgument(
+          hiveOptions.containsParameters(METADATA_LOCATION_PROP),
+          "Table %s is not a valid Iceberg table, metadata location not found",
+          tableName());
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  private void createTable(String newMetadataLocation, TableMetadata metadata) {
+    LOG.debug("Creating a new Iceberg table: {}", tableName());
+    client.createTable(tableName, makeNewTable(metadata, newMetadataLocation));
+  }
+
+  /** Update table properties with concurrent update detection using etag. */
+  private void updateTable(
+      String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) {
+    Table table = client.getTable(tableName);
+    String etag = table.getEtag();
+    Preconditions.checkArgument(
+        !etag.isEmpty(),
+        "Etag of legacy table %s is empty, manually update the table by BigLake API or recreate and retry",
+        tableName());
+    HiveTableOptions options = table.getHiveOptions();
+
+    // If `metadataLocationFromMetastore` is different from metadata location of base, it means
+    // someone has updated metadata location in metastore, which is a conflict update.
+    String metadataLocationFromMetastore =
+        options.getParametersOrDefault(METADATA_LOCATION_PROP, "");

Review Comment:
   This allows the metadata location to be empty in the table. I think it should check for null like we do in the rest of the library. We don't generally fill in an empty string and check for empty when we can check for `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] rdblue commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253529616


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";

Review Comment:
   Should this be `catalog-id` to match the convention used for `project-id`?



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.

Review Comment:
   Is this required or optional?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253612756


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());

Review Comment:
   When is `createNamespace` called with `Namespace.empty()`? Does this need a remote operation or can we just assume that the namespace exists? If it doesn't exist, what does this do on the BigLake side?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253613087


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);

Review Comment:
   In other places, this is `getDatabaseId`. Why not call that 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] rdblue commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253625034


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new NoSuchIcebergTableException(
+            "Table %s is not a valid Iceberg table, metadata location not found", tableName());
+      }
+
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {

Review Comment:
   What throws this besides `NoSuchIcebergTableException`?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253555095


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;

Review Comment:
   Typo: `pulgin` should be `plugin`
   
   Also, this is not a plugin name. It is the catalog name. You can use simply `name`.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;

Review Comment:
   Prefer shorter names, like `io`.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253571157


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());

Review Comment:
   No need to log this. INFO messages should be used to validate normal behavior, but this is part of the config.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261835461


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    mockBigLakeClient = mock(BigLakeClient.class);
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() {
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertEquals(
+        "db_folder/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertEquals(
+        warehouseLocation + "/db.db/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    Exception exception =

Review Comment:
   Done, replaced all.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261828272


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.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.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {

Review Comment:
   This does not work, because checkStyle will return HiddenField error. It requires argument names to be different from class field names.



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

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261877716


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import java.io.File;
+import java.io.IOException;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260365931


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);

Review Comment:
   It is just because we already did the check. Changed to databaseId to be consistent.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260384115


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new NoSuchIcebergTableException(
+            "Table %s is not a valid Iceberg table, metadata location not found", tableName());
+      }
+
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);

Review Comment:
   It never throws because we already checked "hiveOptions.containsParameters(METADATA_LOCATION_PROP)".
   
   I used it because protobuf either getParametersOrThrow or getParametersOrDefault: https://protobuf.dev/reference/java/java-generated/#map-fields



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260347330


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();
+    } else if (namespace.levels().length == 1) {
+      return getMetadata(getDatabase(namespace));
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261470550


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.

Review Comment:
   It happens in an older Spark version: https://github.com/apache/spark/blob/2f3e4e36017d16d67086fd4ecaf39636a2fb4b7c/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala#L253, the listTables check is now removed.
   
   I list all tables in the catalog (all databases) instead.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264604152


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.ArgumentCaptor;
+
+public class BigLakeTableOperationsTest {
+
+  @TempDir private Path temp;
+
+  private static final String CATALOG_NAME = "iceberg";
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier SPARK_TABLE_ID = TableIdentifier.of(DB_ID, TABLE_ID);

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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264755571


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty, returns empty to unblock
+      // deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      bigLakeClient.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return metadata(loadDatabase(namespace));
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return bigLakeProperties == null ? ImmutableMap.of() : bigLakeProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closeableGroup != null) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            bigLakeProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier tableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace namespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName tableName(String dbId, String tableId) {
+    return TableName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId, tableId);
+  }
+
+  private String databaseId(Namespace namespace) {
+    if (namespace.levels().length != 1) {
+      throw new NoSuchNamespaceException(

Review Comment:
   This isn't the right place to throw `NoSuchNamespaceException` because there's no context for the operation that called this. I think this should return either `level(0)` or `null` and let the caller throw exceptions.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264759712


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty, returns empty to unblock
+      // deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {

Review Comment:
   Removed the additional property "location".



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268360832


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()),
+        name.getDatabase());
+  }
+
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getCatalog());
+  }
+
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table getTable(TableName name) {
+    if (name.getTable().isEmpty()) {
+      throw new NoSuchTableException("BigLake API does not allow tables with empty ID");
+    }
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", newName.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table deleteTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.deleteTable(
+                DeleteTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Iterable<Table> listTables(DatabaseName name) {
+    return convertException(
+        () ->
+            stub.listTables(ListTablesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getDatabase());
+  }
+
+  @Override
+  public void close() {
+    stub.close();
+  }
+
+  // Converts BigLake API errors to Iceberg errors.
+  private <T> T convertException(Supplier<T> result, String resourceId) {
+    try {
+      return result.get();
+    } catch (PermissionDeniedException e) {
+      throw new NotAuthorizedException(e, "BigLake API permission denied");

Review Comment:
   Our observation on GCP is that, if we return "permission denied", customers don't know which permission is missing, is it Dataproc or BigLake or Cloud Storage? Having more details here is usually helpful.
   
   But I see other permission denied message does not contain BigLake, I removed "BigLake API" 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1266907784


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockServiceHelper mockServiceHelper;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(),
+            Arrays.<MockGrpcService>asList(new MockMetastoreService()));
+    mockServiceHelper.start();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.PROJECT_ID,
+            GCP_PROJECT,
+            GCPProperties.REGION,
+            GCP_REGION,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            temp.toAbsolutePath().toString());
+
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(mockServiceHelper.createChannelProvider())
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(CATALOG_ID, properties, new BigLakeClient(settings));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(CATALOG_ID, properties, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    if (bigLakeCatalogUsingMockService != null) {
+      bigLakeCatalogUsingMockService.close();
+    }
+
+    if (bigLakeCatalogUsingMockClient != null) {
+      bigLakeCatalogUsingMockClient.close();
+    }
+
+    if (mockServiceHelper != null) {
+      mockServiceHelper.stop();
+    }
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(temp.toAbsolutePath().toString() + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTableToDifferentDatabaseShouldFail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespaceShouldCreateCatalogWhenNamespaceIsEmpty() {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.empty(), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of("n0", "n1"), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenNamespaceIsNotEmpty() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.empty());
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.empty());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenNamespaceIsTooLong() {

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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335233097


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);

Review Comment:
   This can be much simpler and avoid a lot of needless ImmutableList allocation:
   
   ```java
       return dbNames.stream()
           .flatMap(
               dbName ->
                   Streams.stream(client.listTables(dbName))
                       .map(BigLakeCatalog::tableIdentifier))
           .collect(ImmutableList.toImmutableList());
   ```



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335234763


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        String dbId = databaseId(namespace);
+        validateDatabaseId(dbId, namespace);
+        client.deleteDatabase(databaseName(dbId));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> props) {
+    String dbId = databaseId(namespace);
+    validateDatabaseId(dbId, namespace);
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(dbId).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(databaseName(dbId), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> props) {
+    String dbId = databaseId(namespace);
+    validateDatabaseId(dbId, namespace);
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(dbId).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(databaseName(dbId), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls catalog to check existence. BLMS catalog has no metadata today.
+      client.catalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      return loadDatabase(dbId).getHiveOptions().getParametersMap();
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);

Review Comment:
   Here as well, I'd prefer moving this to a check at the beginning:
   
   ```java
     public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
       if (namespace.length() > 1) { // or use isValidNamespace(namespace)
         throw new NoSuchNamespaceException("Namespace does not exist: %s", 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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344450388


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());

Review Comment:
   Calling validation first makes sense. I move all the validations before databaseId(). Depending on whether the input is a table identifier or a namespace, two validation methods checkIdentifier and checkNamespace are used. They are for removing the boilerplate code of throwing exceptions.



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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344492399


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        String dbId = databaseId(namespace);
+        validateDatabaseId(dbId, namespace);
+        client.deleteDatabase(databaseName(dbId));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> props) {
+    String dbId = databaseId(namespace);
+    validateDatabaseId(dbId, namespace);
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(dbId).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(databaseName(dbId), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> props) {
+    String dbId = databaseId(namespace);
+    validateDatabaseId(dbId, namespace);
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(dbId).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(databaseName(dbId), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls catalog to check existence. BLMS catalog has no metadata today.
+      client.catalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      return loadDatabase(dbId).getHiveOptions().getParametersMap();
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", 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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344485731


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268348145


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(

Review Comment:
   We can't tell whether it is permission denied or resource not found from the error. BigLake API returns permission denied for both cases for security considerations.



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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1391491378


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new NoSuchIcebergTableException(
+            "Table %s is not a valid Iceberg table, metadata location not found", tableName());
+      }
+
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);

Review Comment:
   Why not use `getParametersOrDefault` with a null default?



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

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1180618458


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "gcp_location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "blms_catalog";
+
+  public static final String HIVE_METASTORE_WAREHOUSE_DIR = "hive.metastore.warehouse.dir";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileSystem fs;
+  private FileIO fileIO;
+  private Configuration conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    if (!properties.containsKey(PROPERTIES_KEY_GCP_PROJECT)) {
+      throw new ValidationException("GCP project must be specified");
+    }
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+    BigLakeClient newClient;
+    try {
+      newClient =
+          new BigLakeClientImpl(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    if (this.conf == null) {
+      LOG.warn("No Hadoop Configuration was set, using the default environment Configuration");
+      this.conf = new Configuration();
+    }
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) {
+      this.conf.set(
+          HIVE_METASTORE_WAREHOUSE_DIR,
+          LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION)));
+    }
+
+    this.fs =
+        Util.getFs(
+            new Path(
+                LocationUtil.stripTrailingSlash(
+                    properties.get(CatalogProperties.WAREHOUSE_LOCATION))),
+            conf);
+
+    String fileIOImpl =
+        properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.hadoop.HadoopFileIO");
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        conf,
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    client.deleteTable(
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    if (!fromDbId.equals(toDbId)) {
+      throw new ValidationException("New table name must be in the same database");
+    }
+
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {

Review Comment:
   Sorry for the confusion. There are two options: (1) link a `<spark-catalog>` to a physical `<biglake-catalog>`, so the full identifier of a table is just `<spark-catalog>.<database/schema>.<table>`. `CREATE/DROP <spark-catalog>` is supported: it creates/deletes `<biglake-catalog> `via API. (2) Use `<spark-catalog>.<biglake-catalog>.<database/schema>.<table>`.
   
   We choose (1) to avoid the long table identifier in (2) (linking is done by a config `biglake.catalog`). The limitation is that customers can't use two `<biglake-catalog>`s in the same `<spark-catalog>`, they have to install two `<spark-catalog>`s instead. We think it is OK, because <biglake-catalog> is usually an environmental level container (e.g., dev, staging, prod envs), cross env reference is rare.
   
   My concern is, I am not sure whether (1) violates any design pattern of Spark namespaces. Please let me know if it does.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "gcp_location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "blms_catalog";
+
+  public static final String HIVE_METASTORE_WAREHOUSE_DIR = "hive.metastore.warehouse.dir";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileSystem fs;
+  private FileIO fileIO;
+  private Configuration conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    if (!properties.containsKey(PROPERTIES_KEY_GCP_PROJECT)) {
+      throw new ValidationException("GCP project must be specified");
+    }
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+    BigLakeClient newClient;
+    try {
+      newClient =
+          new BigLakeClientImpl(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    if (this.conf == null) {
+      LOG.warn("No Hadoop Configuration was set, using the default environment Configuration");
+      this.conf = new Configuration();
+    }
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) {
+      this.conf.set(
+          HIVE_METASTORE_WAREHOUSE_DIR,
+          LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION)));
+    }
+
+    this.fs =
+        Util.getFs(
+            new Path(
+                LocationUtil.stripTrailingSlash(
+                    properties.get(CatalogProperties.WAREHOUSE_LOCATION))),
+            conf);
+
+    String fileIOImpl =
+        properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.hadoop.HadoopFileIO");
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        conf,
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    client.deleteTable(
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    if (!fromDbId.equals(toDbId)) {
+      throw new ValidationException("New table name must be in the same database");
+    }
+
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {

Review Comment:
   Sorry for the confusion. There are two options: (1) link a `<spark-catalog>` to a physical `<biglake-catalog>`, so the full identifier of a table is just `<spark-catalog>.<database/schema>.<table>`. `CREATE/DROP <spark-catalog>` is supported: it creates/deletes `<biglake-catalog> `via API. (2) Use `<spark-catalog>.<biglake-catalog>.<database/schema>.<table>`.
   
   We choose (1) to avoid the long table identifier in (2) (linking is done by a config `biglake.catalog`). The limitation is that customers can't use two `<biglake-catalog>`s in the same `<spark-catalog>`, they have to install two `<spark-catalog>`s instead. We think it is OK, because `<biglake-catalog>` is usually an environmental level container (e.g., dev, staging, prod envs), cross env reference is rare.
   
   My concern is, I am not sure whether (1) violates any design pattern of Spark namespaces. Please let me know if it does.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1180618458


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "gcp_location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "blms_catalog";
+
+  public static final String HIVE_METASTORE_WAREHOUSE_DIR = "hive.metastore.warehouse.dir";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileSystem fs;
+  private FileIO fileIO;
+  private Configuration conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    if (!properties.containsKey(PROPERTIES_KEY_GCP_PROJECT)) {
+      throw new ValidationException("GCP project must be specified");
+    }
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+    BigLakeClient newClient;
+    try {
+      newClient =
+          new BigLakeClientImpl(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    if (this.conf == null) {
+      LOG.warn("No Hadoop Configuration was set, using the default environment Configuration");
+      this.conf = new Configuration();
+    }
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) {
+      this.conf.set(
+          HIVE_METASTORE_WAREHOUSE_DIR,
+          LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION)));
+    }
+
+    this.fs =
+        Util.getFs(
+            new Path(
+                LocationUtil.stripTrailingSlash(
+                    properties.get(CatalogProperties.WAREHOUSE_LOCATION))),
+            conf);
+
+    String fileIOImpl =
+        properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.hadoop.HadoopFileIO");
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        conf,
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    client.deleteTable(
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    if (!fromDbId.equals(toDbId)) {
+      throw new ValidationException("New table name must be in the same database");
+    }
+
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {

Review Comment:
   Sorry for the confusion. There are two options: (1) link a `<spark-catalog>` to a physical `<biglake-catalog>`, so the full identifier of a table is just `<spark-catalog>.<database/schema>.<table>`. `CREATE/DROP <spark-catalog>` is supported: it creates/deletes `<biglake-catalog> `via API. (2) Use `<spark-catalog>.<biglake-catalog>.<database/schema>.<table>`.
   
   We choose (1) to avoid the long table identifier in (2) (linking is done by a config `biglake.catalog`). The limitation is that customers can't use two `<biglake-catalog>`s in the same `<spark-catalog>`, they have to install two `<spark-catalog>`s instead. We think it is OK, because `<biglake-catalog>` is usually an env level container (e.g., dev, staging, prod envs), cross env reference is rare.
   
   My concern is, I am not sure whether (1) violates any design pattern of Spark namespaces. Please let me know if it does.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1192700398


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeTableOperationsTest {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier SPARK_TABLE_ID = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  @Mock private BigLakeClient bigLakeClient;
+
+  private BigLakeCatalog bigLakeCatalog;
+  private String warehouseLocation;
+  private BigLakeTableOperations tableOps;
+
+  @Before
+  public void before() throws Exception {
+    this.bigLakeCatalog = new BigLakeCatalog();
+    this.warehouseLocation = tempFolder.newFolder("hive-warehouse").toString();
+
+    bigLakeCatalog.setConf(new Configuration());
+    bigLakeCatalog.initialize(
+        CATALOG_ID,
+        /* properties= */ ImmutableMap.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] rdblue commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253602721


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();
+    } else if (namespace.levels().length == 1) {
+      return getMetadata(getDatabase(namespace));
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogPulginName;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return catalogProperties == null ? ImmutableMap.of() : catalogProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  private String getDatabaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            catalogProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier getTableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace getNamespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName getTableName(String dbId, String tableId) {
+    return TableName.of(projectId, location, catalogId, dbId, tableId);
+  }
+
+  private String getDatabaseId(Namespace namespace) {
+    Preconditions.checkArgument(
+        namespace.levels().length == 1,
+        "BigLake database namespace must use format <catalog>.<database>, invalid namespace: %s",
+        namespace);
+    return namespace.level(0);
+  }
+
+  private DatabaseName getDatabaseName(Namespace namespace) {
+    return DatabaseName.of(projectId, location, catalogId, getDatabaseId(namespace));
+  }
+
+  private Database getDatabase(Namespace namespace) {

Review Comment:
   `loadDatabase`?



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260322350


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261879049


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      Preconditions.checkArgument(
+          hiveOptions.containsParameters(METADATA_LOCATION_PROP),
+          "Table %s is not a valid Iceberg table, metadata location not found",
+          tableName());
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  private void createTable(String newMetadataLocation, TableMetadata metadata) {
+    LOG.debug("Creating a new Iceberg table: {}", tableName());
+    client.createTable(tableName, makeNewTable(metadata, newMetadataLocation));
+  }
+
+  /** Update table properties with concurrent update detection using etag. */
+  private void updateTable(
+      String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) {
+    Table table = client.getTable(tableName);
+    String etag = table.getEtag();
+    Preconditions.checkArgument(
+        !etag.isEmpty(),
+        "Etag of legacy table %s is empty, manually update the table by BigLake API or recreate and retry",
+        tableName());
+    HiveTableOptions options = table.getHiveOptions();
+
+    // If `metadataLocationFromMetastore` is different from metadata location of base, it means
+    // someone has updated metadata location in metastore, which is a conflict update.
+    String metadataLocationFromMetastore =
+        options.getParametersOrDefault(METADATA_LOCATION_PROP, "");

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261430453


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260209819


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";

Review Comment:
   Sounds good. I renamed it to region and made it required.



-- 
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] coufon commented on pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1631643837

   Thanks Ryan for the helpful comments. I fixed style issues and will work on the rest comments in another commit.


-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260380872


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new NoSuchIcebergTableException(
+            "Table %s is not a valid Iceberg table, metadata location not found", 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260350099


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();
+    } else if (namespace.levels().length == 1) {
+      return getMetadata(getDatabase(namespace));
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogPulginName;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return catalogProperties == null ? ImmutableMap.of() : catalogProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  private String getDatabaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            catalogProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier getTableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace getNamespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName getTableName(String dbId, String tableId) {
+    return TableName.of(projectId, location, catalogId, dbId, tableId);
+  }
+
+  private String getDatabaseId(Namespace namespace) {
+    Preconditions.checkArgument(
+        namespace.levels().length == 1,
+        "BigLake database namespace must use format <catalog>.<database>, invalid namespace: %s",
+        namespace);
+    return namespace.level(0);
+  }
+
+  private DatabaseName getDatabaseName(Namespace namespace) {
+    return DatabaseName.of(projectId, location, catalogId, getDatabaseId(namespace));
+  }
+
+  private Database getDatabase(Namespace namespace) {
+    return client.getDatabase(getDatabaseName(namespace));
+  }
+
+  private static Map<String, String> getMetadata(Database db) {
+    HiveDatabaseOptions options = db.getHiveOptions();
+    Map<String, String> result = Maps.newHashMap();
+    result.putAll(options.getParameters());
+    result.put("location", options.getLocationUri());
+    return result;
+  }
+
+  private static String invalidNamespaceMessage(Namespace namespace) {
+    return String.format(
+        "BigLake catalog namespace can have zero (catalog) or one level (database), invalid"
+            + " namespace: %s",

Review Comment:
   Sure, 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261837387


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    mockBigLakeClient = mock(BigLakeClient.class);
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() {
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertEquals(
+        "db_folder/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertEquals(
+        warehouseLocation + "/db.db/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    Exception exception =
+        assertThrows(
+            IllegalArgumentException.class,
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")));
+    assertEquals(
+        "Cannot rename table db0.t1 to db1.t2: database must match", exception.getMessage());
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {

Review Comment:
   Done, changed all.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264585025


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    mockBigLakeClient = mock(BigLakeClient.class);
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() {
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertEquals(
+        "db_folder/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertEquals(
+        warehouseLocation + "/db.db/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    Exception exception =
+        assertThrows(
+            IllegalArgumentException.class,
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")));
+    assertEquals(
+        "Cannot rename table db0.t1 to db1.t2: database must match", exception.getMessage());
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {

Review Comment:
   Sorry I missed some because of starting the cleanup in middle. 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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264750328


##########
gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java:
##########
@@ -36,6 +37,19 @@ public class GCPProperties implements Serializable {
   public static final String GCS_CHANNEL_READ_CHUNK_SIZE = "gcs.channel.read.chunk-size-bytes";
   public static final String GCS_CHANNEL_WRITE_CHUNK_SIZE = "gcs.channel.write.chunk-size-bytes";
 
+  // For BigQuery BigLake Metastore.
+  // The endpoint of BigLake API.
+  // Optional, default to BigLakeCatalog.DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String BIGLAKE_PROJECT_ID = "biglake.project-id";

Review Comment:
   Is this specific to BigLake? The comment makes it sound like it's a GCP project, not a BigLake project?
   
   In general, I'm wondering if these all require a `biglake.` prefix. We could probably use the more standard `uri`, then it doesn't seem like a bad idea to have catalog config properties like `project-id`, `region`, and `catalog-id`.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1266093108


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.ArgumentCaptor;
+
+public class BigLakeTableOperationsTest {
+
+  @TempDir private Path temp;
+
+  private static final String CATALOG_NAME = "iceberg";
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  private static final Schema SCHEMA =
+      new Schema(
+          required(1, "id", Types.IntegerType.get(), "unique ID"),
+          required(2, "data", Types.StringType.get()));
+
+  private BigLakeClient bigLakeClient = mock(BigLakeClient.class);
+  private BigLakeCatalog bigLakeCatalog;
+  private BigLakeTableOperations tableOps;
+
+  @BeforeEach
+  public void before() throws Exception {
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.PROJECT_ID,
+            GCP_PROJECT,
+            GCPProperties.REGION,
+            GCP_REGION,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            temp.toFile().getAbsolutePath(),
+            GCPProperties.BIGLAKE_CATALOG_ID,
+            CATALOG_ID);
+
+    bigLakeCatalog = new BigLakeCatalog();
+    bigLakeCatalog.setConf(new Configuration());
+    bigLakeCatalog.initialize(CATALOG_NAME, properties, bigLakeClient);
+    tableOps = (BigLakeTableOperations) bigLakeCatalog.newTableOps(TABLE_IDENTIFIER);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalog.close();
+  }
+
+  @Test
+  public void testDoCommitShouldUseEtagForUpdateTable() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    Table createdTable = createTestTable();
+
+    Table tableWithEtag = createdTable.toBuilder().setEtag("etag").build();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(tableWithEtag, tableWithEtag);
+
+    org.apache.iceberg.Table loadedTable = bigLakeCatalog.loadTable(TABLE_IDENTIFIER);
+
+    when(bigLakeClient.updateTableParameters(any(), any(), any())).thenReturn(tableWithEtag);
+    loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit();
+
+    ArgumentCaptor<TableName> nameCaptor = ArgumentCaptor.forClass(TableName.class);
+    ArgumentCaptor<String> etagCaptor = ArgumentCaptor.forClass(String.class);
+    verify(bigLakeClient, times(1))
+        .updateTableParameters(nameCaptor.capture(), any(), etagCaptor.capture());
+    assertThat(nameCaptor.getValue()).isEqualTo(TABLE_NAME);
+    assertThat(etagCaptor.getValue()).isEqualTo("etag");
+  }
+
+  @Test
+  public void testDoCommitShouldFailWhenEtagMismatch() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    Table createdTable = createTestTable();
+
+    Table tableWithEtag = createdTable.toBuilder().setEtag("etag").build();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(tableWithEtag, tableWithEtag);
+
+    org.apache.iceberg.Table loadedTable = bigLakeCatalog.loadTable(TABLE_IDENTIFIER);
+
+    when(bigLakeClient.updateTableParameters(any(), any(), any()))
+        .thenThrow(
+            new AbortedException(
+                new RuntimeException("error message etag mismatch"),
+                GrpcStatusCode.of(Code.ABORTED),
+                false));
+
+    assertThatThrownBy(
+            () -> loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit())
+        .isInstanceOf(CommitFailedException.class);
+  }
+
+  @Test
+  public void testDoFreshRefreshShouldReturnNullForNonIcebergTable() {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenReturn(Table.newBuilder().setName(TABLE_NAME.toString()).build());
+
+    assertThat(tableOps.refresh()).isNull();
+  }
+
+  @Test
+  public void testTableName() {
+    assertThat(tableOps.tableName()).isEqualTo("iceberg.db.tbl");
+  }
+
+  private Table createTestTable() throws IOException {
+    TableIdentifier tableIdent =
+        TableIdentifier.of(TABLE_NAME.getDatabase(), TABLE_NAME.getTable());
+    String tableDir =
+        new File(temp.toFile().getAbsolutePath(), TABLE_NAME.getTable()).getAbsolutePath();
+
+    bigLakeCatalog
+        .buildTable(tableIdent, SCHEMA)
+        .withLocation(tableDir)
+        .createTransaction()
+        .commitTransaction();
+
+    Optional<String> metadataLocation = getAnyIcebergMetadataFilePath(tableDir);

Review Comment:
   It is for getting the JSON metadata file path. I renamed the method to be clearer. Any JSON file works since the table is just created.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264607385


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              projectId,
+              region);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      bigLakeClient.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return metadata(loadDatabase(namespace));
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return bigLakeProperties == null ? ImmutableMap.of() : bigLakeProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    closeableGroup.close();
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            bigLakeProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier tableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace namespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName tableName(String dbId, String tableId) {
+    return TableName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId, tableId);
+  }
+
+  private String databaseId(Namespace namespace) {
+    if (namespace.levels().length != 1) {
+      throw new NoSuchNamespaceException(
+          "BigLake database namespace must use format <catalog>.<database>, invalid namespace: %s",
+          namespace);
+    }
+
+    return namespace.level(0);
+  }
+
+  private DatabaseName databaseName(Namespace namespace) {
+    return DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, databaseId(namespace));
+  }
+
+  private Database loadDatabase(Namespace namespace) {
+    return bigLakeClient.getDatabase(databaseName(namespace));
+  }
+
+  private static Map<String, String> metadata(Database db) {
+    HiveDatabaseOptions options = db.getHiveOptions();
+    return new ImmutableMap.Builder<String, String>()
+        .putAll(options.getParameters())
+        .put("location", options.getLocationUri())

Review Comment:
   It is for users to inspect database location URI. It follows the behavior of Hive metastore catalog.



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

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264753224


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.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.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {

Review Comment:
   Sound good, done.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.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.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {

Review Comment:
   Sounds good, 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264761919


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty, returns empty to unblock
+      // deletion.
+      return ImmutableList.of();

Review Comment:
   It should return empty list for database namespace, and throw NoSuchNamespaceException if the namespace is invalid (neither a catalog nor database). Updated 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264759393


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)

Review Comment:
   Good point. Adding this hardcoded property "location" may cause conflict. I removed 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264756873


##########
gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java:
##########
@@ -36,6 +37,19 @@ public class GCPProperties implements Serializable {
   public static final String GCS_CHANNEL_READ_CHUNK_SIZE = "gcs.channel.read.chunk-size-bytes";
   public static final String GCS_CHANNEL_WRITE_CHUNK_SIZE = "gcs.channel.write.chunk-size-bytes";
 
+  // For BigQuery BigLake Metastore.
+  // The endpoint of BigLake API.
+  // Optional, default to BigLakeCatalog.DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String BIGLAKE_ENDPOINT = "biglake.endpoint";

Review Comment:
   Changed to CatalogProperties.URI.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1266912120


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {

Review Comment:
   Done.



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO io;
+  // The name of this Iceberg catalog plugin.
+  private final String name;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO io, String name, TableName tableName) {
+    this.client = client;
+    this.io = io;
+    this.name = name;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new NoSuchIcebergTableException(
+            "Invalid Iceberg table %s: missing metadata location", tableName());
+      }
+
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268368896


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()),
+        name.getDatabase());
+  }
+
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getCatalog());
+  }
+
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table getTable(TableName name) {
+    if (name.getTable().isEmpty()) {
+      throw new NoSuchTableException("BigLake API does not allow tables with empty ID");
+    }
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {

Review Comment:
   Because this one returns "table already exists", convertException handles "namespace already exists". I can also combine them and don't distinguish table/namespace in the error message. Please let me know if you prefer this way.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335233952


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));

Review Comment:
   Prefer throwing exceptions at the start of methods during validation, rather than mixing validation and core logic.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335233598


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);

Review Comment:
   As I mentioned above, I think that using `validateDatabaseId` makes this code harder to understand. Rather than throwing inside another method, I think it's best to use a boolean method to validate the namespace expectation (e.g., `isValid`) and throw the right exception inline so that it is clear what is being thrown.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335236017


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Catalog already exists: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Catalog catalog(CatalogName name) {
+    try {
+      return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+    } catch (PermissionDeniedException e) {

Review Comment:
   I think it would make sense if everything returned 404, but returning 401/403 and translating that to 404 doesn't make any sense to me.
   
   When we last talked, the argument for this behavior was that returning 401/403 leaks the fact that the object exists. But this is actually doing the opposite and hiding the fact that the object doesn't exist by throwing `NoSuchNamespaceException`. I guess that in the end, the service is returning a single response for all 401/403/404 cases, but it's strange to use a permission error rather than a not-exists error.
   
   Can you confirm that the service will never return a 404?
   
   Also, if the service guarantees that 401, 403, and 404 will result in `PermissionDeniedException`, then that should be documented in this class somewhere, probably in class-level Javadoc. Then maybe we don't need to add the confusing "(or permission denied)" to all of the messages?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335236411


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Catalog already exists: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Catalog catalog(CatalogName name) {
+    try {
+      return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+    } catch (PermissionDeniedException e) {
+      throw new NoSuchNamespaceException(
+          e, "Catalog does not exist: %s (or permission denied)", name.getCatalog());
+    }
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    try {
+      stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+    } catch (PermissionDeniedException e) {
+      throw new NoSuchNamespaceException(
+          e, "Catalog does not exist: %s (or permission denied)", name.getCatalog());
+    }
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {

Review Comment:
   Shouldn't this also handle `PermissionDeniedException` since that may be thrown by any of the calls? Here it should probably be translated to Iceberg's `ForbiddenException` or `NotAuthorizedException`. Or is this an appropriate place to return `NoSuchNamespaceException` to indicate that the catalog doesn't exist?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1211937918


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+
+/** A client interface of Google BigLake service. */
+interface BigLakeClient {

Review Comment:
   I'm still struggling to see how this interface makes sense. The implementation only wraps `MetastoreServiceClient` so it seems like you should be able to mock or spy that instead of creating one 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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344490651


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        String dbId = databaseId(namespace);
+        validateDatabaseId(dbId, namespace);
+        client.deleteDatabase(databaseName(dbId));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);

Review Comment:
   Removed the warning.



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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344513347


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Catalog already exists: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Catalog catalog(CatalogName name) {
+    try {
+      return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+    } catch (PermissionDeniedException e) {
+      throw new NoSuchNamespaceException(
+          e, "Catalog does not exist: %s (or permission denied)", name.getCatalog());
+    }
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    try {
+      stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+    } catch (PermissionDeniedException e) {
+      throw new NoSuchNamespaceException(
+          e, "Catalog does not exist: %s (or permission denied)", name.getCatalog());
+    }
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {

Review Comment:
   Yes, it is a good point. We should check whether it failed due to parent not found or permission denied. It needs parsing the error message. I added a TODO to do it in a follow-up PR, to avoid adding more new code to this PR which has been around for long.



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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344525502


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        String dbId = databaseId(namespace);
+        validateDatabaseId(dbId, namespace);
+        client.deleteDatabase(databaseName(dbId));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);

Review Comment:
   But we can't tell whether it is not found or permission denied. For permission denied, we did want to notice the user to double check. Maybe the BigLake client should not convert 403 to 404, instead just convert 403 to Iceberg NotAuthorizedException? then the downstream never treat these errors as not found.



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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344423119


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");

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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344511751


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Catalog already exists: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Catalog catalog(CatalogName name) {
+    try {
+      return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+    } catch (PermissionDeniedException e) {
+      throw new NoSuchNamespaceException(
+          e, "Catalog does not exist: %s (or permission denied)", name.getCatalog());
+    }
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    try {
+      stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+    } catch (PermissionDeniedException e) {
+      throw new NoSuchNamespaceException(
+          e, "Catalog does not exist: %s (or permission denied)", name.getCatalog());
+    }
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {

Review Comment:
   The wrapper convertException will handle PermissionDeniedException and convert it to Iceberg's NotAuthorizedException. It is extracted to a wrapper for reducing boilerplate code.
   



-- 
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] coufon commented on pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1527835917

   > Zhou Fang can you help explain and document how the atomic update works? I assume this is somehow related to the `etag` and `updateTable` call, but it's not entirely clear how the atomic swap is enforced.
   
   Thank you so much for the review. I added a comment in the code to explain atomic update:
   
         // Updating a BLMS table with etag. The BLMS server transactionally (1) checks that the etag
         // of a table on server is the same as the etag provided by the client, and (2) update the
         // table (and its etag). The server returns an error containing message "etag mismatch", if
         // the etag on server has changed.


-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1192722279


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      Preconditions.checkArgument(
+          hiveOptions.containsParameters(METADATA_LOCATION_PROP),
+          "Table %s is not a valid Iceberg table, metadata location not found",
+          tableName());
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  private void createTable(String newMetadataLocation, TableMetadata metadata) {
+    LOG.debug("Creating a new Iceberg table: {}", tableName());
+    client.createTable(tableName, makeNewTable(metadata, newMetadataLocation));
+  }
+
+  /** Update table properties with concurrent update detection using etag. */
+  private void updateTable(
+      String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) {
+    Table table = client.getTable(tableName);
+    String etag = table.getEtag();
+    Preconditions.checkArgument(
+        !etag.isEmpty(),
+        "Etag of legacy table %s is empty, manually update the table by BigLake API or recreate and retry",
+        tableName());
+    HiveTableOptions options = table.getHiveOptions();
+
+    // If `metadataLocationFromMetastore` is different from metadata location of base, it means
+    // someone has updated metadata location in metastore, which is a conflict update.
+    String metadataLocationFromMetastore =
+        options.getParametersOrDefault(METADATA_LOCATION_PROP, "");

Review Comment:
   Protobuf Java only have getFooOrDefault and getFooOrThrow. I kept getParametersOrDefault and do an empty string check immediately. It is the same as checking nullness. Catching an error or returning a map from proto + get value are more verbose. Please let me know whether it works.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1186927542


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeTableOperationsTest {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier SPARK_TABLE_ID = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  @Mock private BigLakeClient bigLakeClient;
+
+  private BigLakeCatalog bigLakeCatalog;
+  private String warehouseLocation;
+  private BigLakeTableOperations tableOps;
+
+  @Before
+  public void before() throws Exception {
+    this.bigLakeCatalog = new BigLakeCatalog();
+    this.warehouseLocation = tempFolder.newFolder("hive-warehouse").toString();
+
+    bigLakeCatalog.setConf(new Configuration());
+    bigLakeCatalog.initialize(
+        CATALOG_ID,
+        /* properties= */ ImmutableMap.of(

Review Comment:
   Can you move this to a separate variable instead?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1186929252


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      Preconditions.checkArgument(
+          hiveOptions.containsParameters(METADATA_LOCATION_PROP),
+          "Table %s is not a valid Iceberg table, metadata location not found",
+          tableName());
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  private void createTable(String newMetadataLocation, TableMetadata metadata) {
+    LOG.debug("Creating a new Iceberg table: {}", tableName());
+    client.createTable(tableName, makeNewTable(metadata, newMetadataLocation));
+  }
+
+  /** Update table properties with concurrent update detection using etag. */
+  private void updateTable(
+      String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) {
+    Table table = client.getTable(tableName);
+    String etag = table.getEtag();
+    Preconditions.checkArgument(
+        !etag.isEmpty(),
+        "Etag of legacy table %s is empty, manually update the table by BigLake API or recreate and retry",
+        tableName());
+    HiveTableOptions options = table.getHiveOptions();
+
+    // If `metadataLocationFromMetastore` is different from metadata location of base, it means
+    // someone has updated metadata location in metastore, which is a conflict update.
+    String metadataLocationFromMetastore =
+        options.getParametersOrDefault(METADATA_LOCATION_PROP, "");
+    if (!metadataLocationFromMetastore.isEmpty()
+        && !metadataLocationFromMetastore.equals(oldMetadataLocation)) {
+      throw new CommitFailedException(
+          "Base metadata location '%s' is not same as the current table metadata location '%s' for"
+              + " %s.%s",
+          oldMetadataLocation,
+          metadataLocationFromMetastore,
+          tableName.getDatabase(),
+          tableName.getTable());
+    }
+
+    try {
+      // Updating a BLMS table with etag. The BLMS server transactionally (1) checks that the etag
+      // of a table on server is the same as the etag provided by the client, and (2) updates the
+      // table (and its etag). The server returns an error containing message "etag mismatch", if
+      // the etag on server has changed.
+      client.updateTableParameters(
+          tableName, buildTableParameters(newMetadataLocation, metadata), etag);

Review Comment:
   Just to clarify: the service considers the etag an assertion and that's what gives us an atomic swap, right?



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

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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253528775


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.

Review Comment:
   Do you intend to do this in the current PR or wait for some reason?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253605280


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();
+    } else if (namespace.levels().length == 1) {
+      return getMetadata(getDatabase(namespace));
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogPulginName;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return catalogProperties == null ? ImmutableMap.of() : catalogProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  private String getDatabaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            catalogProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier getTableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace getNamespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName getTableName(String dbId, String tableId) {
+    return TableName.of(projectId, location, catalogId, dbId, tableId);
+  }
+
+  private String getDatabaseId(Namespace namespace) {
+    Preconditions.checkArgument(
+        namespace.levels().length == 1,
+        "BigLake database namespace must use format <catalog>.<database>, invalid namespace: %s",
+        namespace);
+    return namespace.level(0);
+  }
+
+  private DatabaseName getDatabaseName(Namespace namespace) {
+    return DatabaseName.of(projectId, location, catalogId, getDatabaseId(namespace));
+  }
+
+  private Database getDatabase(Namespace namespace) {
+    return client.getDatabase(getDatabaseName(namespace));
+  }
+
+  private static Map<String, String> getMetadata(Database db) {
+    HiveDatabaseOptions options = db.getHiveOptions();
+    Map<String, String> result = Maps.newHashMap();
+    result.putAll(options.getParameters());
+    result.put("location", options.getLocationUri());
+    return result;
+  }
+
+  private static String invalidNamespaceMessage(Namespace namespace) {
+    return String.format(
+        "BigLake catalog namespace can have zero (catalog) or one level (database), invalid"
+            + " namespace: %s",

Review Comment:
   Can you make this shorter?
   
   Also, this is only used in places that have already checked for a 0-length namespace. I think this only needs to worry about namespaces that are too long.
   ```java
   return String.format("Invalid namespace (too long): %s", 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] rdblue commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253602396


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();
+    } else if (namespace.levels().length == 1) {
+      return getMetadata(getDatabase(namespace));
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogPulginName;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return catalogProperties == null ? ImmutableMap.of() : catalogProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  private String getDatabaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            catalogProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier getTableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace getNamespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName getTableName(String dbId, String tableId) {
+    return TableName.of(projectId, location, catalogId, dbId, tableId);
+  }
+
+  private String getDatabaseId(Namespace namespace) {
+    Preconditions.checkArgument(

Review Comment:
   This isn't the right exception. If the namespace is longer than 1, then it should throw `NoSuchNamespaceException`. If the namespace is empty, then this should just return null, I 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


[GitHub] [iceberg] rdblue commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253597608


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));

Review Comment:
   Passing an empty string is always suspicious. I think the idea is to get `BigLakeTableOperations` to throw `NoSuchTableException` later after checking the namespace here.
   
   I think it's a better pattern to override `isValidIdentifier` so that this check is done before calling `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] rdblue commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253598177


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));

Review Comment:
   Why the `tableId` comment?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253626214


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new NoSuchIcebergTableException(
+            "Table %s is not a valid Iceberg table, metadata location not found", tableName());
+      }
+
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (AlreadyExistsException | CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());

Review Comment:
   This should be the catalog name that was passed to `BigLakeCatalog`, not the catalog ID for BigLake.



-- 
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] coufon commented on pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1585089810

   > Running CI
   
   There is a style check failure. 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261783555


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,

Review Comment:
   I know why I added `prod` after running checkstyleMain. It does not allow arguments to have the same name as fields. I added BigLake prefix to fields instead. 



-- 
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] coufon commented on pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1633410320

   Thanks @rdblue and @nastra for the review. I have addressed the 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] nastra commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1262401578


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    mockBigLakeClient = mock(BigLakeClient.class);
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() {
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertEquals(
+        "db_folder/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertEquals(
+        warehouseLocation + "/db.db/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    Exception exception =
+        assertThrows(
+            IllegalArgumentException.class,
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")));
+    assertEquals(
+        "Cannot rename table db0.t1 to db1.t2: database must match", exception.getMessage());
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {

Review Comment:
   there seem to be still a few left, like `testDefaultWarehouseWithDatabaseLocation_asExpected` and some others



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260385182


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new NoSuchIcebergTableException(
+            "Table %s is not a valid Iceberg table, metadata location not found", tableName());
+      }
+
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {

Review Comment:
   This is thrown by the RPC "client.getTable(tableName)". It happens when this table is not found on BigLake Metastore server.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260377052


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {

Review Comment:
   It is because invalid namespace throws IllegalArgumentException. I have changed it to throw NoSuchNamespaceException and removed this catch.



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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344463833


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);

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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335232821


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();

Review Comment:
   The RHS is not used and should be omitted.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335234104


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty, returns empty to unblock
+      // deletion.
+      return ImmutableList.of();

Review Comment:
   I think that this needs to throw `NoSuchNamespaceException` if the database doesn't exist.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268250707


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(

Review Comment:
   This looks incorrect to me. If permission was denied, then the database does exist. This should not hide that.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268246925


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,404 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    return new BigLakeTableOperations(client, io, name(), tableName(db, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(db) : locationUri, identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else if (namespace.levels().length == 1) {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    } else {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(db, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    if (fromDb == null) {
+      throwInvalidDbNamespaceError(from.namespace());
+    }
+
+    String toDb = databaseId(to.namespace());
+    if (toDb == null) {
+      throwInvalidDbNamespaceError(to.namespace());
+    }
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> props) {
+    DatabaseName dbName = databaseName(namespace);
+    if (dbName == null) {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(dbName, optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> props) {
+    DatabaseName dbName = databaseName(namespace);
+    if (dbName == null) {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(dbName, optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return loadDatabase(namespace).getHiveOptions().getParametersMap();
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return properties == null ? ImmutableMap.of() : properties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closeableGroup != null) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    return String.format("%s/%s.db", warehouseLocation, dbId);
+  }
+
+  private static TableIdentifier tableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace namespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName tableName(String dbId, String tableId) {
+    return TableName.of(projectId, region, catalogId, dbId, tableId);
+  }
+
+  private String databaseId(Namespace namespace) {
+    return namespace.levels().length == 1 ? namespace.level(0) : null;
+  }
+
+  private DatabaseName databaseName(Namespace namespace) {
+    String db = databaseId(namespace);
+    return db == null ? null : DatabaseName.of(projectId, region, catalogId, db);
+  }
+
+  private Database loadDatabase(Namespace namespace) {
+    return client.getDatabase(databaseName(namespace));
+  }
+
+  private void throwInvalidDbNamespaceError(Namespace namespace) {
+    throw new NoSuchNamespaceException(
+        "Invalid BigLake database namespace: %s", namespace.isEmpty() ? "empty" : namespace);

Review Comment:
   No need for "BigLake database" here. Just "Invalid namespace" is enough.



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

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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268250707


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(

Review Comment:
   This looks incorrect to me. If permission was denied, then the catalog does exist. This should not hide that.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264750291


##########
gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java:
##########
@@ -36,6 +37,19 @@ public class GCPProperties implements Serializable {
   public static final String GCS_CHANNEL_READ_CHUNK_SIZE = "gcs.channel.read.chunk-size-bytes";
   public static final String GCS_CHANNEL_WRITE_CHUNK_SIZE = "gcs.channel.write.chunk-size-bytes";
 
+  // For BigQuery BigLake Metastore.
+  // The endpoint of BigLake API.
+  // Optional, default to BigLakeCatalog.DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String BIGLAKE_ENDPOINT = "biglake.endpoint";

Review Comment:
   Why not use the standard catalog property `uri`?



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264585287


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.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.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient {
+
+  private final String projectId;
+  private final String region;

Review Comment:
   Removed them.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264587119


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264604260


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              projectId,
+              region);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      bigLakeClient.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return metadata(loadDatabase(namespace));
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return bigLakeProperties == null ? ImmutableMap.of() : bigLakeProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    closeableGroup.close();
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            bigLakeProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier tableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace namespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName tableName(String dbId, String tableId) {
+    return TableName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId, tableId);
+  }
+
+  private String databaseId(Namespace namespace) {
+    if (namespace.levels().length != 1) {
+      throw new NoSuchNamespaceException(
+          "BigLake database namespace must use format <catalog>.<database>, invalid namespace: %s",
+          namespace);
+    }
+
+    return namespace.level(0);
+  }
+
+  private DatabaseName databaseName(Namespace namespace) {
+    return DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, databaseId(namespace));
+  }
+
+  private Database loadDatabase(Namespace namespace) {
+    return bigLakeClient.getDatabase(databaseName(namespace));
+  }
+
+  private static Map<String, String> metadata(Database db) {
+    HiveDatabaseOptions options = db.getHiveOptions();
+    return new ImmutableMap.Builder<String, String>()
+        .putAll(options.getParameters())

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264604475


##########
build.gradle:
##########
@@ -608,18 +608,42 @@ project(':iceberg-gcp') {
 
     implementation platform('com.google.cloud:libraries-bom')
     implementation 'com.google.cloud:google-cloud-storage'
-
-    testImplementation 'com.google.cloud:google-cloud-nio'
+    implementation 'com.google.cloud:google-cloud-biglake'
+    compileOnly('org.apache.hadoop:hadoop-common') {
+      exclude group: 'org.apache.avro', module: 'avro'
+      exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+      exclude group: 'javax.servlet', module: 'servlet-api'
+      exclude group: 'com.google.code.gson', module: 'gson'
+    }
+    compileOnly("org.apache.hive:hive-metastore") {

Review Comment:
   Nice catch, thanks. The internal version uses it to implement Hive Metastore federation (https://cloud.google.com/bigquery/docs/manage-open-source-metadata#copy-iceberg-table), I forgot to clean it up. Removed 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] rdblue commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264755633


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty, returns empty to unblock
+      // deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    bigLakeClient.updateDatabaseParameters(
+        databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      bigLakeClient.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return metadata(loadDatabase(namespace));
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return bigLakeProperties == null ? ImmutableMap.of() : bigLakeProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closeableGroup != null) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            bigLakeProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier tableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace namespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName tableName(String dbId, String tableId) {
+    return TableName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId, tableId);
+  }
+
+  private String databaseId(Namespace namespace) {
+    if (namespace.levels().length != 1) {
+      throw new NoSuchNamespaceException(
+          namespace.isEmpty()
+              ? "Invalid BigLake database namespace: empty"
+              : String.format(
+                  "BigLake database namespace must use format <catalog>.<database>, invalid namespace: %s",
+                  namespace));
+    }
+
+    return namespace.level(0);
+  }
+
+  private DatabaseName databaseName(Namespace namespace) {
+    return DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, databaseId(namespace));

Review Comment:
   With the refactor of `databaseId`, this should also return `null` if the ID is 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264755913


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);

Review Comment:
   The name is just for making field and args different. Renamed it to properties.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1265346290


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockServiceHelper mockServiceHelper;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(),
+            Arrays.<MockGrpcService>asList(new MockMetastoreService()));
+    mockServiceHelper.start();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.PROJECT_ID,
+            GCP_PROJECT,
+            GCPProperties.REGION,
+            GCP_REGION,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            temp.toAbsolutePath().toString());
+
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(mockServiceHelper.createChannelProvider())
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(CATALOG_ID, properties, new BigLakeClient(settings));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(CATALOG_ID, properties, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    if (bigLakeCatalogUsingMockService != null) {
+      bigLakeCatalogUsingMockService.close();
+    }
+
+    if (bigLakeCatalogUsingMockClient != null) {
+      bigLakeCatalogUsingMockClient.close();
+    }
+
+    if (mockServiceHelper != null) {
+      mockServiceHelper.stop();
+    }
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(temp.toAbsolutePath().toString() + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTableToDifferentDatabaseShouldFail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespaceShouldCreateCatalogWhenNamespaceIsEmpty() {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.empty(), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of("n0", "n1"), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenNamespaceIsNotEmpty() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.empty());
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.empty());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenNamespaceIsTooLong() {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of("n0", "n1"))).isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() {

Review Comment:
   ```suggestion
     public void testSetPropertiesShouldFailWhenNamespaceIsInvalid() {
   ```



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockServiceHelper mockServiceHelper;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(),
+            Arrays.<MockGrpcService>asList(new MockMetastoreService()));
+    mockServiceHelper.start();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.PROJECT_ID,
+            GCP_PROJECT,
+            GCPProperties.REGION,
+            GCP_REGION,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            temp.toAbsolutePath().toString());
+
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(mockServiceHelper.createChannelProvider())
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(CATALOG_ID, properties, new BigLakeClient(settings));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(CATALOG_ID, properties, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    if (bigLakeCatalogUsingMockService != null) {
+      bigLakeCatalogUsingMockService.close();
+    }
+
+    if (bigLakeCatalogUsingMockClient != null) {
+      bigLakeCatalogUsingMockClient.close();
+    }
+
+    if (mockServiceHelper != null) {
+      mockServiceHelper.stop();
+    }
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(temp.toAbsolutePath().toString() + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTableToDifferentDatabaseShouldFail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespaceShouldCreateCatalogWhenNamespaceIsEmpty() {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.empty(), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of("n0", "n1"), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenNamespaceIsNotEmpty() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.empty());
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.empty());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenNamespaceIsTooLong() {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of("n0", "n1"))).isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() {
+    assertThatThrownBy(
+            () -> bigLakeCatalogUsingMockClient.setProperties(Namespace.empty(), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class)
+        .hasMessage("Invalid BigLake database namespace: empty");
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of("db", "tbl"), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class)
+        .hasMessage("Invalid BigLake database namespace: db.tbl");
+  }
+
+  @Test
+  public void testSetPropertiesShouldSucceedForDatabase() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.setProperties(
+                Namespace.of("db"), ImmutableMap.of("key2", "value222", "key3", "value3")))
+        .isTrue();
+    verify(mockBigLakeClient, times(1))
+        .updateDatabaseParameters(
+            DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db"),
+            ImmutableMap.of("key1", "value1", "key2", "value222", "key3", "value3"));
+  }
+
+  @Test
+  public void testRemovePropertiesShouldFailWhenNamespacesAreInvalid() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.removeProperties(
+                    Namespace.empty(), ImmutableSet.of()))
+        .isInstanceOf(NoSuchNamespaceException.class)
+        .hasMessage("Invalid BigLake database namespace: empty");
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.removeProperties(
+                    Namespace.of("db", "tbl"), ImmutableSet.of()))
+        .isInstanceOf(NoSuchNamespaceException.class)
+        .hasMessage("Invalid BigLake database namespace: db.tbl");
+  }
+
+  @Test
+  public void testRemovePropertiesShouldSucceedForDatabase() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.removeProperties(
+                Namespace.of("db"), ImmutableSet.of("key1", "key3")))
+        .isTrue();
+    verify(mockBigLakeClient, times(1))
+        .updateDatabaseParameters(
+            DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db"),
+            ImmutableMap.of("key2", "value2"));
+  }
+
+  @Test
+  public void testEmptyNamespaceLoadsCatalogMetadata() {
+    assertThat(bigLakeCatalogUsingMockClient.loadNamespaceMetadata(Namespace.empty())).isEmpty();
+    verify(mockBigLakeClient, times(1))
+        .getCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testLoadNamespaceMetadataForDatabases() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .setLocationUri("my location uri")
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertThat(bigLakeCatalogUsingMockClient.loadNamespaceMetadata(Namespace.of("db")))
+        .containsAllEntriesOf(ImmutableMap.of("key1", "value1", "key2", "value2"));
+  }
+
+  @Test
+  public void testLoadNamespaceMetadataShouldFailWhenInvalid() {
+    assertThatThrownBy(
+            () -> bigLakeCatalogUsingMockClient.loadNamespaceMetadata(Namespace.of("n0", "n1")))
+        .isInstanceOf(NoSuchNamespaceException.class)
+        .hasMessage("Namespace does not exist: n0.n1");
+  }
+
+  @Test
+  public void testNewTableOpsShouldfailedForInvalidNamespace() {

Review Comment:
   ```suggestion
     public void testNewTableOpsShouldFailForInvalidNamespace() {
   ```



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.ArgumentCaptor;
+
+public class BigLakeTableOperationsTest {
+
+  @TempDir private Path temp;
+
+  private static final String CATALOG_NAME = "iceberg";
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  private static final Schema SCHEMA =
+      new Schema(
+          required(1, "id", Types.IntegerType.get(), "unique ID"),
+          required(2, "data", Types.StringType.get()));
+
+  private BigLakeClient bigLakeClient = mock(BigLakeClient.class);
+  private BigLakeCatalog bigLakeCatalog;
+  private BigLakeTableOperations tableOps;
+
+  @BeforeEach
+  public void before() throws Exception {
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.PROJECT_ID,
+            GCP_PROJECT,
+            GCPProperties.REGION,
+            GCP_REGION,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            temp.toFile().getAbsolutePath(),
+            GCPProperties.BIGLAKE_CATALOG_ID,
+            CATALOG_ID);
+
+    bigLakeCatalog = new BigLakeCatalog();
+    bigLakeCatalog.setConf(new Configuration());
+    bigLakeCatalog.initialize(CATALOG_NAME, properties, bigLakeClient);
+    tableOps = (BigLakeTableOperations) bigLakeCatalog.newTableOps(TABLE_IDENTIFIER);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalog.close();
+  }
+
+  @Test
+  public void testDoCommitShouldUseEtagForUpdateTable() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    Table createdTable = createTestTable();
+
+    Table tableWithEtag = createdTable.toBuilder().setEtag("etag").build();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(tableWithEtag, tableWithEtag);
+
+    org.apache.iceberg.Table loadedTable = bigLakeCatalog.loadTable(TABLE_IDENTIFIER);
+
+    when(bigLakeClient.updateTableParameters(any(), any(), any())).thenReturn(tableWithEtag);
+    loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit();
+
+    ArgumentCaptor<TableName> nameCaptor = ArgumentCaptor.forClass(TableName.class);
+    ArgumentCaptor<String> etagCaptor = ArgumentCaptor.forClass(String.class);
+    verify(bigLakeClient, times(1))
+        .updateTableParameters(nameCaptor.capture(), any(), etagCaptor.capture());
+    assertThat(nameCaptor.getValue()).isEqualTo(TABLE_NAME);
+    assertThat(etagCaptor.getValue()).isEqualTo("etag");
+  }
+
+  @Test
+  public void testDoCommitShouldFailWhenEtagMismatch() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    Table createdTable = createTestTable();
+
+    Table tableWithEtag = createdTable.toBuilder().setEtag("etag").build();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(tableWithEtag, tableWithEtag);
+
+    org.apache.iceberg.Table loadedTable = bigLakeCatalog.loadTable(TABLE_IDENTIFIER);
+
+    when(bigLakeClient.updateTableParameters(any(), any(), any()))
+        .thenThrow(
+            new AbortedException(
+                new RuntimeException("error message etag mismatch"),
+                GrpcStatusCode.of(Code.ABORTED),
+                false));
+
+    assertThatThrownBy(
+            () -> loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit())
+        .isInstanceOf(CommitFailedException.class);
+  }
+
+  @Test
+  public void testDoFreshRefreshShouldReturnNullForNonIcebergTable() {

Review Comment:
   is this called on purpose a "fresh refresh"? Maybe something like "initial refresh"?



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkNotNull(initClient, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    return new BigLakeTableOperations(client, io, name(), tableName(db, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(db) : locationUri, identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else if (namespace.levels().length == 1) {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    } else {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String db = databaseId(identifier.namespace());
+    if (db == null) {
+      throwInvalidDbNamespaceError(identifier.namespace());
+    }
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(db, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    if (fromDb == null) {
+      throwInvalidDbNamespaceError(from.namespace());
+    }
+
+    String toDb = databaseId(to.namespace());
+    if (toDb == null) {
+      throwInvalidDbNamespaceError(to.namespace());
+    }
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> props) {
+    DatabaseName dbName = databaseName(namespace);
+    if (dbName == null) {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(dbName, optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> props) {
+    DatabaseName dbName = databaseName(namespace);
+    if (dbName == null) {
+      throwInvalidDbNamespaceError(namespace);
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(dbName, optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      return loadDatabase(namespace).getHiveOptions().getParametersMap();
+    } else {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return properties == null ? ImmutableMap.of() : properties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closeableGroup != null) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
+    return tableIdentifier.namespace().levels().length == 1;
+  }
+
+  private String databaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");

Review Comment:
   same as above, throwing an IAE is slightly more descriptive to the user seeing this error



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockServiceHelper mockServiceHelper;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(),
+            Arrays.<MockGrpcService>asList(new MockMetastoreService()));
+    mockServiceHelper.start();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.PROJECT_ID,
+            GCP_PROJECT,
+            GCPProperties.REGION,
+            GCP_REGION,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            temp.toAbsolutePath().toString());
+
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(mockServiceHelper.createChannelProvider())
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(CATALOG_ID, properties, new BigLakeClient(settings));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(CATALOG_ID, properties, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    if (bigLakeCatalogUsingMockService != null) {
+      bigLakeCatalogUsingMockService.close();
+    }
+
+    if (bigLakeCatalogUsingMockClient != null) {
+      bigLakeCatalogUsingMockClient.close();
+    }
+
+    if (mockServiceHelper != null) {
+      mockServiceHelper.stop();
+    }
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(temp.toAbsolutePath().toString() + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTableToDifferentDatabaseShouldFail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespaceShouldCreateCatalogWhenNamespaceIsEmpty() {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.empty(), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of("n0", "n1"), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenNamespaceIsNotEmpty() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.empty());
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.empty());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenNamespaceIsTooLong() {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of("n0", "n1"))).isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() {
+    assertThatThrownBy(
+            () -> bigLakeCatalogUsingMockClient.setProperties(Namespace.empty(), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class)
+        .hasMessage("Invalid BigLake database namespace: empty");
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of("db", "tbl"), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class)
+        .hasMessage("Invalid BigLake database namespace: db.tbl");
+  }
+
+  @Test
+  public void testSetPropertiesShouldSucceedForDatabase() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.setProperties(
+                Namespace.of("db"), ImmutableMap.of("key2", "value222", "key3", "value3")))
+        .isTrue();
+    verify(mockBigLakeClient, times(1))
+        .updateDatabaseParameters(
+            DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db"),
+            ImmutableMap.of("key1", "value1", "key2", "value222", "key3", "value3"));
+  }
+
+  @Test
+  public void testRemovePropertiesShouldFailWhenNamespacesAreInvalid() {

Review Comment:
   ```suggestion
     public void testRemovePropertiesShouldFailWhenNamespaceIsInvalid() {
   ```



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.ArgumentCaptor;
+
+public class BigLakeTableOperationsTest {
+
+  @TempDir private Path temp;
+
+  private static final String CATALOG_NAME = "iceberg";
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  private static final Schema SCHEMA =
+      new Schema(
+          required(1, "id", Types.IntegerType.get(), "unique ID"),
+          required(2, "data", Types.StringType.get()));
+
+  private BigLakeClient bigLakeClient = mock(BigLakeClient.class);
+  private BigLakeCatalog bigLakeCatalog;
+  private BigLakeTableOperations tableOps;
+
+  @BeforeEach
+  public void before() throws Exception {
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.PROJECT_ID,
+            GCP_PROJECT,
+            GCPProperties.REGION,
+            GCP_REGION,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            temp.toFile().getAbsolutePath(),
+            GCPProperties.BIGLAKE_CATALOG_ID,
+            CATALOG_ID);
+
+    bigLakeCatalog = new BigLakeCatalog();
+    bigLakeCatalog.setConf(new Configuration());
+    bigLakeCatalog.initialize(CATALOG_NAME, properties, bigLakeClient);
+    tableOps = (BigLakeTableOperations) bigLakeCatalog.newTableOps(TABLE_IDENTIFIER);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalog.close();
+  }
+
+  @Test
+  public void testDoCommitShouldUseEtagForUpdateTable() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    Table createdTable = createTestTable();
+
+    Table tableWithEtag = createdTable.toBuilder().setEtag("etag").build();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(tableWithEtag, tableWithEtag);
+
+    org.apache.iceberg.Table loadedTable = bigLakeCatalog.loadTable(TABLE_IDENTIFIER);
+
+    when(bigLakeClient.updateTableParameters(any(), any(), any())).thenReturn(tableWithEtag);
+    loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit();
+
+    ArgumentCaptor<TableName> nameCaptor = ArgumentCaptor.forClass(TableName.class);
+    ArgumentCaptor<String> etagCaptor = ArgumentCaptor.forClass(String.class);
+    verify(bigLakeClient, times(1))
+        .updateTableParameters(nameCaptor.capture(), any(), etagCaptor.capture());
+    assertThat(nameCaptor.getValue()).isEqualTo(TABLE_NAME);
+    assertThat(etagCaptor.getValue()).isEqualTo("etag");
+  }
+
+  @Test
+  public void testDoCommitShouldFailWhenEtagMismatch() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    Table createdTable = createTestTable();
+
+    Table tableWithEtag = createdTable.toBuilder().setEtag("etag").build();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(tableWithEtag, tableWithEtag);
+
+    org.apache.iceberg.Table loadedTable = bigLakeCatalog.loadTable(TABLE_IDENTIFIER);
+
+    when(bigLakeClient.updateTableParameters(any(), any(), any()))
+        .thenThrow(
+            new AbortedException(
+                new RuntimeException("error message etag mismatch"),
+                GrpcStatusCode.of(Code.ABORTED),
+                false));
+
+    assertThatThrownBy(
+            () -> loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit())
+        .isInstanceOf(CommitFailedException.class);

Review Comment:
   this should have a `.hasMessage(...)` check



##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.ArgumentCaptor;
+
+public class BigLakeTableOperationsTest {
+
+  @TempDir private Path temp;
+
+  private static final String CATALOG_NAME = "iceberg";
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  private static final Schema SCHEMA =
+      new Schema(
+          required(1, "id", Types.IntegerType.get(), "unique ID"),
+          required(2, "data", Types.StringType.get()));
+
+  private BigLakeClient bigLakeClient = mock(BigLakeClient.class);
+  private BigLakeCatalog bigLakeCatalog;
+  private BigLakeTableOperations tableOps;
+
+  @BeforeEach
+  public void before() throws Exception {
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.PROJECT_ID,
+            GCP_PROJECT,
+            GCPProperties.REGION,
+            GCP_REGION,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            temp.toFile().getAbsolutePath(),
+            GCPProperties.BIGLAKE_CATALOG_ID,
+            CATALOG_ID);
+
+    bigLakeCatalog = new BigLakeCatalog();
+    bigLakeCatalog.setConf(new Configuration());
+    bigLakeCatalog.initialize(CATALOG_NAME, properties, bigLakeClient);
+    tableOps = (BigLakeTableOperations) bigLakeCatalog.newTableOps(TABLE_IDENTIFIER);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalog.close();
+  }
+
+  @Test
+  public void testDoCommitShouldUseEtagForUpdateTable() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    Table createdTable = createTestTable();
+
+    Table tableWithEtag = createdTable.toBuilder().setEtag("etag").build();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(tableWithEtag, tableWithEtag);
+
+    org.apache.iceberg.Table loadedTable = bigLakeCatalog.loadTable(TABLE_IDENTIFIER);
+
+    when(bigLakeClient.updateTableParameters(any(), any(), any())).thenReturn(tableWithEtag);
+    loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit();
+
+    ArgumentCaptor<TableName> nameCaptor = ArgumentCaptor.forClass(TableName.class);
+    ArgumentCaptor<String> etagCaptor = ArgumentCaptor.forClass(String.class);
+    verify(bigLakeClient, times(1))
+        .updateTableParameters(nameCaptor.capture(), any(), etagCaptor.capture());
+    assertThat(nameCaptor.getValue()).isEqualTo(TABLE_NAME);
+    assertThat(etagCaptor.getValue()).isEqualTo("etag");
+  }
+
+  @Test
+  public void testDoCommitShouldFailWhenEtagMismatch() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    Table createdTable = createTestTable();
+
+    Table tableWithEtag = createdTable.toBuilder().setEtag("etag").build();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(tableWithEtag, tableWithEtag);
+
+    org.apache.iceberg.Table loadedTable = bigLakeCatalog.loadTable(TABLE_IDENTIFIER);
+
+    when(bigLakeClient.updateTableParameters(any(), any(), any()))
+        .thenThrow(
+            new AbortedException(
+                new RuntimeException("error message etag mismatch"),
+                GrpcStatusCode.of(Code.ABORTED),
+                false));
+
+    assertThatThrownBy(
+            () -> loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit())
+        .isInstanceOf(CommitFailedException.class);
+  }
+
+  @Test
+  public void testDoFreshRefreshShouldReturnNullForNonIcebergTable() {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenReturn(Table.newBuilder().setName(TABLE_NAME.toString()).build());
+
+    assertThat(tableOps.refresh()).isNull();
+  }
+
+  @Test
+  public void testTableName() {
+    assertThat(tableOps.tableName()).isEqualTo("iceberg.db.tbl");
+  }
+
+  private Table createTestTable() throws IOException {
+    TableIdentifier tableIdent =
+        TableIdentifier.of(TABLE_NAME.getDatabase(), TABLE_NAME.getTable());
+    String tableDir =
+        new File(temp.toFile().getAbsolutePath(), TABLE_NAME.getTable()).getAbsolutePath();
+
+    bigLakeCatalog
+        .buildTable(tableIdent, SCHEMA)
+        .withLocation(tableDir)
+        .createTransaction()
+        .commitTransaction();
+
+    Optional<String> metadataLocation = getAnyIcebergMetadataFilePath(tableDir);

Review Comment:
   this seems confusing, could you elaborate please why this is necessary?



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkNotNull(initClient, "BigLake client must not be null");

Review Comment:
   Iceberg typically uses `Preconditions.checkArgument(initClient != null, "...")` because this is slightly clearer to an end user instead of the typical NPE



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261872574


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1192724413


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      Preconditions.checkArgument(
+          hiveOptions.containsParameters(METADATA_LOCATION_PROP),
+          "Table %s is not a valid Iceberg table, metadata location not found",
+          tableName());
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  private void createTable(String newMetadataLocation, TableMetadata metadata) {
+    LOG.debug("Creating a new Iceberg table: {}", tableName());
+    client.createTable(tableName, makeNewTable(metadata, newMetadataLocation));
+  }
+
+  /** Update table properties with concurrent update detection using etag. */
+  private void updateTable(
+      String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) {
+    Table table = client.getTable(tableName);
+    String etag = table.getEtag();
+    Preconditions.checkArgument(
+        !etag.isEmpty(),
+        "Etag of legacy table %s is empty, manually update the table by BigLake API or recreate and retry",
+        tableName());
+    HiveTableOptions options = table.getHiveOptions();
+
+    // If `metadataLocationFromMetastore` is different from metadata location of base, it means
+    // someone has updated metadata location in metastore, which is a conflict update.
+    String metadataLocationFromMetastore =
+        options.getParametersOrDefault(METADATA_LOCATION_PROP, "");
+    if (!metadataLocationFromMetastore.isEmpty()
+        && !metadataLocationFromMetastore.equals(oldMetadataLocation)) {
+      throw new CommitFailedException(
+          "Base metadata location '%s' is not same as the current table metadata location '%s' for"
+              + " %s.%s",
+          oldMetadataLocation,
+          metadataLocationFromMetastore,
+          tableName.getDatabase(),
+          tableName.getTable());
+    }
+
+    try {
+      // Updating a BLMS table with etag. The BLMS server transactionally (1) checks that the etag
+      // of a table on server is the same as the etag provided by the client, and (2) updates the
+      // table (and its etag). The server returns an error containing message "etag mismatch", if
+      // the etag on server has changed.
+      client.updateTableParameters(
+          tableName, buildTableParameters(newMetadataLocation, metadata), etag);
+    } catch (AbortedException e) {
+      if (e.getMessage().toLowerCase().contains("etag mismatch")) {
+        throw new CommitFailedException(
+            "Updating table failed due to conflict updates (etag mismatch)");
+      }
+    }
+  }
+
+  private Table makeNewTable(TableMetadata metadata, String metadataFileLocation) {
+    Table.Builder builder = Table.newBuilder().setType(Table.Type.HIVE);
+    builder
+        .getHiveOptionsBuilder()
+        .setTableType("EXTERNAL_TABLE")
+        .setStorageDescriptor(
+            StorageDescriptor.newBuilder()
+                .setLocationUri(metadata.location())
+                .setInputFormat("org.apache.hadoop.mapred.FileInputFormat")
+                .setOutputFormat("org.apache.hadoop.mapred.FileOutputFormat")

Review Comment:
   Yes, we hope a HMS proxy can return Iceberg tables from BigLake API as they were from HMS, so engines supporting HMS can use BigLake. In the BigLake API, such tables have type Hive. In future we may add Iceberg type and store only metadata needed for Iceberg.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1187955905


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";

Review Comment:
   I don't think we should defer the property updates to a separate PR.  It doesn't make sense to introduce them here just to move them later.  They are also public fields which means that if they end up in a release, they will need to go through a deprecation cycle.  We can add the additional properties to the GCPProperties and make sure that they are namespaced appropriately to work with other properties.
   
   If we can determine the project id from the bucket itself, that would be great, but one of the major points of the GCSFileIO is to remove dependencies on hadoop/hdfs, so using that connector is not a solution for this.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1192702898


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeTableOperationsTest {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier SPARK_TABLE_ID = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  @Mock private BigLakeClient bigLakeClient;
+
+  private BigLakeCatalog bigLakeCatalog;
+  private String warehouseLocation;
+  private BigLakeTableOperations tableOps;
+
+  @Before
+  public void before() throws Exception {
+    this.bigLakeCatalog = new BigLakeCatalog();
+    this.warehouseLocation = tempFolder.newFolder("hive-warehouse").toString();
+
+    bigLakeCatalog.setConf(new Configuration());
+    bigLakeCatalog.initialize(
+        CATALOG_ID,
+        /* properties= */ ImmutableMap.of(
+            BigLakeCatalog.PROPERTIES_KEY_GCP_PROJECT,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation),
+        GCP_PROJECT,
+        GCP_REGION,
+        bigLakeClient);
+    this.tableOps = (BigLakeTableOperations) bigLakeCatalog.newTableOps(SPARK_TABLE_ID);
+  }
+
+  @Test
+  public void testDoFresh_fetchLatestMetadataFromBigLake() throws Exception {
+    Table createdTable = createTestTable();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(createdTable);
+
+    tableOps.refresh();
+    assertEquals(
+        createdTable
+            .getHiveOptions()
+            .getParametersOrDefault(BigLakeTestUtils.METADATA_LOCATION_PROP, ""),
+        tableOps.currentMetadataLocation());
+
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    // Refresh fails when table is not found but metadata already presents.
+    assertThrows(NoSuchTableException.class, () -> tableOps.refresh());
+  }
+
+  @Test
+  public void testDoFresh_failForNonIcebergTable() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenReturn(Table.newBuilder().setName(TABLE_NAME.toString()).build());
+
+    Exception exception = assertThrows(IllegalArgumentException.class, () -> tableOps.refresh());
+    assertTrue(exception.getMessage().contains("metadata location not found"));
+  }
+
+  @Test
+  public void testDoFresh_noOpWhenMetadataAndTableNotFound() throws Exception {

Review Comment:
   Has udpated BigLakeCatalogTest to use CatalogTests.



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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "dchristle (via GitHub)" <gi...@apache.org>.
dchristle commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1781549984

   > We released these code here (https://cloud.google.com/bigquery/docs/manage-open-source-metadata#connect-dataproc-vm):
   > 
   > Iceberg 1.2.0: gs://spark-lib/biglake/biglake-catalog-iceberg1.2.0-0.1.1-with-dependencies.jar Iceberg 0.14.0: gs://spark-lib/biglake/biglake-catalog-iceberg0.14.0-0.1.1-with-dependencies.jar
   > 
   > Feel free to try these before this PR is merged.
   
   @coufon @emkornfield Are there instructions on how to build the equivalent artifacts using this pull request, including dependencies (`with-dependencies`) & any shading? It would be nice to use a jar built with 1.4.x as a dependency when using Iceberg 1.4.x. 


-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335234524


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        String dbId = databaseId(namespace);
+        validateDatabaseId(dbId, namespace);
+        client.deleteDatabase(databaseName(dbId));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> props) {
+    String dbId = databaseId(namespace);
+    validateDatabaseId(dbId, namespace);
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(dbId).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(databaseName(dbId), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> props) {
+    String dbId = databaseId(namespace);
+    validateDatabaseId(dbId, namespace);
+
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(dbId).toBuilder().getHiveOptionsBuilder();
+    props.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(databaseName(dbId), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      // Calls catalog to check existence. BLMS catalog has no metadata today.
+      client.catalog(catalogName);
+      return ImmutableMap.of();
+    } else if (namespace.levels().length == 1) {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      return loadDatabase(dbId).getHiveOptions().getParametersMap();

Review Comment:
   Is this parameter map immutable or unmodifiable?



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


Re: [PR] GCP: Add Iceberg Catalog for GCP BigLake Metastore [iceberg]

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1344458998


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260218415


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260338985


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();
+    } else if (namespace.levels().length == 1) {
+      return getMetadata(getDatabase(namespace));
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogPulginName;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return catalogProperties == null ? ImmutableMap.of() : catalogProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  private String getDatabaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            catalogProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier getTableIdentifier(Table table) {
+    TableName tableName = TableName.parse(table.getName());
+    return TableIdentifier.of(Namespace.of(tableName.getDatabase()), tableName.getTable());
+  }
+
+  private static Namespace getNamespace(Database db) {
+    return Namespace.of(DatabaseName.parse(db.getName()).getDatabase());
+  }
+
+  private TableName getTableName(String dbId, String tableId) {
+    return TableName.of(projectId, location, catalogId, dbId, tableId);
+  }
+
+  private String getDatabaseId(Namespace namespace) {
+    Preconditions.checkArgument(
+        namespace.levels().length == 1,
+        "BigLake database namespace must use format <catalog>.<database>, invalid namespace: %s",
+        namespace);
+    return namespace.level(0);
+  }
+
+  private DatabaseName getDatabaseName(Namespace namespace) {
+    return DatabaseName.of(projectId, location, catalogId, getDatabaseId(namespace));
+  }
+
+  private Database getDatabase(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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264585508


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO io;
+  // The catalog name.
+  private final String name;

Review Comment:
   CatalogName can mean two things here, <catalog_plugin> in "spark.sql.catalog.<catalog_plugin>" or <catalog_id> specified by "spark.sql.catalog.<catalog_plugin>.biglake.catalog-id". The first one is the virtual name of this catalog plugin, the second one is the real resource name of BigLake catalog. Ryan suggested to just call the first one "name" in an earlier comment. I extended the comment to make it clear.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260346550


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260355561


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261827580


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,410 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.api.gax.rpc.InvalidArgumentException;
+import com.google.api.pathtemplate.ValidationException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            BigLakeCatalog.PROPERTIES_KEY_GCP_PROJECT,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    mockBigLakeClient = mock(BigLakeClient.class);
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() {
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testNamespaceWithSlash() {
+    BigLakeCatalog catalog = catalog();
+
+    Exception exception =
+        assertThrows(
+            InvalidArgumentException.class, () -> catalog.createNamespace(Namespace.of("new/db")));
+    assertTrue(exception.getMessage().contains("Database ID is invalid"));
+  }
+
+  @Test
+  public void testTableNameWithSlash() {
+    BigLakeCatalog catalog = catalog();
+
+    catalog.createNamespace(Namespace.of("ns"));
+    TableIdentifier ident = TableIdentifier.of("ns", "tab/le");
+
+    Exception exception =
+        assertThrows(ValidationException.class, () -> catalog.buildTable(ident, SCHEMA).create());
+    assertEquals("Invalid character \"/\" in path section \"tab/le\".", exception.getMessage());
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertEquals(
+        "db_folder/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertEquals(
+        warehouseLocation + "/db.db/table",
+        bigLakeCatalogUsingMockClient.defaultWarehouseLocation(TableIdentifier.of("db", "table")));
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    Exception exception =
+        assertThrows(
+            IllegalArgumentException.class,
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")));
+    assertEquals("New table name must be in the same database", exception.getMessage());
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespace_failWhenInvalid() throws Exception {
+    Exception exception =
+        assertThrows(
+            IllegalArgumentException.class,
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()));
+    assertEquals(
+        "BigLake catalog namespace can have zero (catalog) or one level (database), invalid"
+            + " namespace: n0.n1",
+        exception.getMessage());
+  }
+
+  @Test
+  public void testListNamespaces_emptyWhenInvalid() {
+    assertTrue(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db")).isEmpty());
+  }
+
+  @Test
+  public void testDropNamespace_deleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  // BigLake catalog plugin supports dropping a BigLake catalog resource. Spark calls listTables
+  // with an empty namespace in this case, the purpose is verifying the namespace is empty. We
+  // check whether there are databases in the BigLake catalog instead.
+  @Test
+  public void testListTables_emptyNamespace_noDatabase() {
+    when(mockBigLakeClient.listDatabases(any(CatalogName.class))).thenReturn(ImmutableList.of());
+
+    assertTrue(bigLakeCatalogUsingMockClient.listTables(Namespace.of()).isEmpty());
+    verify(mockBigLakeClient, times(1))
+        .listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTables_emptyNamespace_checkCatalogEmptiness() {
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(ImmutableList.of(Database.getDefaultInstance()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertEquals(1, result.size());
+    assertEquals(TableIdentifier.of("placeholder"), result.get(0));
+  }
+
+  @Test
+  public void testDropNamespace_failWhenInvalid() throws Exception {
+    Exception exception =
+        assertThrows(
+            IllegalArgumentException.class,
+            () ->
+                bigLakeCatalogUsingMockClient.dropNamespace(
+                    Namespace.of(new String[] {"n0", "n1"})));
+    assertEquals(
+        "BigLake catalog namespace can have zero (catalog) or one level (database), invalid"
+            + " namespace: n0.n1",
+        exception.getMessage());
+  }
+
+  @Test
+  public void testSetProperties_failWhenNamespacesAreInvalid() throws Exception {
+    assertFalse(
+        bigLakeCatalogUsingMockClient.setProperties(
+            Namespace.of(new String[] {}), ImmutableMap.of()));
+    assertFalse(
+        bigLakeCatalogUsingMockClient.setProperties(
+            Namespace.of(new String[] {"db", "tbl"}), ImmutableMap.of()));
+  }
+
+  @Test
+  public void testSetProperties_succeedForDatabase() throws Exception {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertTrue(
+        bigLakeCatalogUsingMockClient.setProperties(
+            Namespace.of(new String[] {"db"}),
+            ImmutableMap.of("key2", "value222", "key3", "value3")));
+    verify(mockBigLakeClient, times(1))
+        .updateDatabaseParameters(
+            DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db"),
+            ImmutableMap.of("key1", "value1", "key2", "value222", "key3", "value3"));
+  }
+
+  @Test
+  public void testRemoveProperties_failWhenNamespacesAreInvalid() throws Exception {
+    assertFalse(
+        bigLakeCatalogUsingMockClient.removeProperties(
+            Namespace.of(new String[] {}), ImmutableSet.of()));
+    assertFalse(
+        bigLakeCatalogUsingMockClient.removeProperties(
+            Namespace.of(new String[] {"db", "tbl"}), ImmutableSet.of()));
+  }
+
+  @Test
+  public void testRemoveProperties_succeedForDatabase() throws Exception {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertTrue(
+        bigLakeCatalogUsingMockClient.removeProperties(
+            Namespace.of(new String[] {"db"}), ImmutableSet.of("key1", "key3")));
+    verify(mockBigLakeClient, times(1))
+        .updateDatabaseParameters(
+            DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db"),
+            ImmutableMap.of("key2", "value2"));
+  }
+
+  @Test
+  public void testLoadNamespaceMetadata_catalogAsExpected() throws Exception {
+    assertTrue(
+        bigLakeCatalogUsingMockClient
+            .loadNamespaceMetadata(Namespace.of(new String[] {}))
+            .isEmpty());
+    verify(mockBigLakeClient, times(1))
+        .getCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testLoadNamespaceMetadata_databaseAsExpected() throws Exception {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .setLocationUri("my location uri")
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertEquals(
+        ImmutableMap.of("location", "my location uri", "key1", "value1", "key2", "value2"),
+        bigLakeCatalogUsingMockClient.loadNamespaceMetadata(Namespace.of(new String[] {"db"})));
+  }
+
+  @Test
+  public void testLoadNamespaceMetadata_failWhenInvalid() throws Exception {
+    Exception exception =
+        assertThrows(
+            IllegalArgumentException.class,
+            () ->
+                bigLakeCatalogUsingMockClient.loadNamespaceMetadata(
+                    Namespace.of(new String[] {"n0", "n1"})));
+    assertEquals(
+        "BigLake catalog namespace can have zero (catalog) or one level (database), invalid"
+            + " namespace: n0.n1",
+        exception.getMessage());
+  }
+
+  @Test
+  public void testSetBigLakeCatalogInProperties_asExpected() throws Exception {

Review Comment:
   Removed this test. To test the `initialize` called by Spark without issuing a real RPC to BigLake service, we need to let the catalog accept a gRPC channel, but this is only used for test. It may not be a good idea to add this path to make the catalog more complex.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261786309


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {

Review Comment:
   Same HiddenField issue from checkStyle. I renamed the class field to bigLakeClient.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260391573


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,410 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.api.gax.rpc.InvalidArgumentException;
+import com.google.api.pathtemplate.ValidationException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            BigLakeCatalog.PROPERTIES_KEY_GCP_PROJECT,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    mockBigLakeClient = mock(BigLakeClient.class);
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() {
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testNamespaceWithSlash() {

Review Comment:
   I removed the slash tests.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264751588


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;

Review Comment:
   When you need to have argument names that differ from field names, rename the arguments, not the fields. The fields are used everywhere so it's much more awkward to have long names prefixed by `bigLake`.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264750328


##########
gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java:
##########
@@ -36,6 +37,19 @@ public class GCPProperties implements Serializable {
   public static final String GCS_CHANNEL_READ_CHUNK_SIZE = "gcs.channel.read.chunk-size-bytes";
   public static final String GCS_CHANNEL_WRITE_CHUNK_SIZE = "gcs.channel.write.chunk-size-bytes";
 
+  // For BigQuery BigLake Metastore.
+  // The endpoint of BigLake API.
+  // Optional, default to BigLakeCatalog.DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String BIGLAKE_PROJECT_ID = "biglake.project-id";

Review Comment:
   Is this specific to BigLake? The comment makes it sound like it's a GCP project, not a BigLake project?



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1266084044


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkNotNull(initClient, "BigLake client must not be null");

Review Comment:
   I don't understand this one, I think the purpose of checkNotNull is for simplifying "Preconditions.checkArgument(initClient != null, ...)"  ? I also find checkNotNull has been used a lot in other classes.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1272920818


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()),
+        name.getDatabase());
+  }
+
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getCatalog());
+  }
+
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table getTable(TableName name) {
+    if (name.getTable().isEmpty()) {
+      throw new NoSuchTableException("BigLake API does not allow tables with empty ID");
+    }
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {

Review Comment:
   Removed convertException that takes no effect.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264755763


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty, returns empty to unblock
+      // deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {

Review Comment:
   I think this is unnecessary with the refactor of `databaseName`.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264755875


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty, returns empty to unblock
+      // deletion.
+      return ImmutableList.of();

Review Comment:
   Shouldn't this check whether the namespace exists? If not then it should throw `NoSuchNamespaceException`.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268348639


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(

Review Comment:
   I updated all "not found" error message to "not found or permission denied".



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1180618458


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "gcp_location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "blms_catalog";
+
+  public static final String HIVE_METASTORE_WAREHOUSE_DIR = "hive.metastore.warehouse.dir";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileSystem fs;
+  private FileIO fileIO;
+  private Configuration conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    if (!properties.containsKey(PROPERTIES_KEY_GCP_PROJECT)) {
+      throw new ValidationException("GCP project must be specified");
+    }
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+    BigLakeClient newClient;
+    try {
+      newClient =
+          new BigLakeClientImpl(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    if (this.conf == null) {
+      LOG.warn("No Hadoop Configuration was set, using the default environment Configuration");
+      this.conf = new Configuration();
+    }
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) {
+      this.conf.set(
+          HIVE_METASTORE_WAREHOUSE_DIR,
+          LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION)));
+    }
+
+    this.fs =
+        Util.getFs(
+            new Path(
+                LocationUtil.stripTrailingSlash(
+                    properties.get(CatalogProperties.WAREHOUSE_LOCATION))),
+            conf);
+
+    String fileIOImpl =
+        properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.hadoop.HadoopFileIO");
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        conf,
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    client.deleteTable(
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    if (!fromDbId.equals(toDbId)) {
+      throw new ValidationException("New table name must be in the same database");
+    }
+
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {

Review Comment:
   Sorry for the confusion. There are two options: (1) link a `<spark-catalog>` to a physical `<biglake-catalog>`, so the full identifier of a table is just `<spark-catalog>.<database/schema>.<table>`. `CREATE/DROP <spark-catalog>` is supported: it creates/deletes `<biglake-catalog> `via API. (2) Use `<spark-catalog>.<biglake-catalog>.<database/schema>.<table>`.
   
   We choose (1) to avoid the long table identifier in (2) (linking is done by a config `biglake.catalog`). The limitation is that customers can't use two `<biglake-catalog>`s in the same `<spark-catalog>`, they have to install two `<spark-catalog>`s instead. We think it is OK, because `<biglake-catalog>` is usually an env level container (e.g., use different BigLake catalogs for dev, staging, prod envs), cross env reference is rare.
   
   My concern is, I am not sure whether (1) violates any design pattern of Spark namespaces. Please let me know if it does.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1180622721


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {

Review Comment:
   Updated it. Thanks!



##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+
+/** A client interface of Google BigLake service. */
+public interface BigLakeClient {

Review Comment:
   Updated 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] rdblue commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1186927852


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeTableOperationsTest {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier SPARK_TABLE_ID = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  @Mock private BigLakeClient bigLakeClient;
+
+  private BigLakeCatalog bigLakeCatalog;
+  private String warehouseLocation;
+  private BigLakeTableOperations tableOps;
+
+  @Before
+  public void before() throws Exception {
+    this.bigLakeCatalog = new BigLakeCatalog();
+    this.warehouseLocation = tempFolder.newFolder("hive-warehouse").toString();
+
+    bigLakeCatalog.setConf(new Configuration());
+    bigLakeCatalog.initialize(
+        CATALOG_ID,
+        /* properties= */ ImmutableMap.of(
+            BigLakeCatalog.PROPERTIES_KEY_GCP_PROJECT,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation),
+        GCP_PROJECT,
+        GCP_REGION,
+        bigLakeClient);
+    this.tableOps = (BigLakeTableOperations) bigLakeCatalog.newTableOps(SPARK_TABLE_ID);
+  }
+
+  @Test
+  public void testDoFresh_fetchLatestMetadataFromBigLake() throws Exception {
+    Table createdTable = createTestTable();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(createdTable);
+
+    tableOps.refresh();
+    assertEquals(
+        createdTable
+            .getHiveOptions()
+            .getParametersOrDefault(BigLakeTestUtils.METADATA_LOCATION_PROP, ""),
+        tableOps.currentMetadataLocation());
+
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    // Refresh fails when table is not found but metadata already presents.
+    assertThrows(NoSuchTableException.class, () -> tableOps.refresh());
+  }
+
+  @Test
+  public void testDoFresh_failForNonIcebergTable() throws Exception {
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenReturn(Table.newBuilder().setName(TABLE_NAME.toString()).build());
+
+    Exception exception = assertThrows(IllegalArgumentException.class, () -> tableOps.refresh());
+    assertTrue(exception.getMessage().contains("metadata location not found"));
+  }
+
+  @Test
+  public void testDoFresh_noOpWhenMetadataAndTableNotFound() throws Exception {

Review Comment:
   Looks like a lot of these tests violate the catalog API, so I'll take a look after this implements `CatalogTests`.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1186927672


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeTableOperationsTest {

Review Comment:
   Can you extend `CatalogTests` that has the standard catalog behavior tests?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1186929149


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      Preconditions.checkArgument(
+          hiveOptions.containsParameters(METADATA_LOCATION_PROP),
+          "Table %s is not a valid Iceberg table, metadata location not found",
+          tableName());
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.
+        throw e;
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation);
+  }
+
+  // The doCommit method should provide implementation on how to update with metadata location
+  // atomically
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    boolean isNewTable = base == null;
+    String newMetadataLocation = writeNewMetadataIfRequired(isNewTable, metadata);
+
+    CommitStatus commitStatus = CommitStatus.FAILURE;
+    try {
+      if (isNewTable) {
+        createTable(newMetadataLocation, metadata);
+      } else {
+        updateTable(base.metadataFileLocation(), newMetadataLocation, metadata);
+      }
+      commitStatus = CommitStatus.SUCCESS;
+    } catch (CommitFailedException | CommitStateUnknownException e) {
+      throw e;
+    } catch (Throwable e) {
+      commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+      if (commitStatus == CommitStatus.FAILURE) {
+        throw new CommitFailedException(e, "Failed to commit");
+      }
+      if (commitStatus == CommitStatus.UNKNOWN) {
+        throw new CommitStateUnknownException(e);
+      }
+    } finally {
+      try {
+        if (commitStatus == CommitStatus.FAILURE) {
+          LOG.warn("Failed to commit updates to table {}", tableName());
+          io().deleteFile(newMetadataLocation);
+        }
+      } catch (RuntimeException e) {
+        LOG.error(
+            "Failed to cleanup metadata file at {} for table {}",
+            newMetadataLocation,
+            tableName(),
+            e);
+      }
+    }
+  }
+
+  @Override
+  public String tableName() {
+    return String.format(
+        "%s.%s.%s", tableName.getCatalog(), tableName.getDatabase(), tableName.getTable());
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  private void createTable(String newMetadataLocation, TableMetadata metadata) {
+    LOG.debug("Creating a new Iceberg table: {}", tableName());
+    client.createTable(tableName, makeNewTable(metadata, newMetadataLocation));
+  }
+
+  /** Update table properties with concurrent update detection using etag. */
+  private void updateTable(
+      String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) {
+    Table table = client.getTable(tableName);
+    String etag = table.getEtag();
+    Preconditions.checkArgument(
+        !etag.isEmpty(),
+        "Etag of legacy table %s is empty, manually update the table by BigLake API or recreate and retry",
+        tableName());
+    HiveTableOptions options = table.getHiveOptions();
+
+    // If `metadataLocationFromMetastore` is different from metadata location of base, it means
+    // someone has updated metadata location in metastore, which is a conflict update.
+    String metadataLocationFromMetastore =
+        options.getParametersOrDefault(METADATA_LOCATION_PROP, "");
+    if (!metadataLocationFromMetastore.isEmpty()
+        && !metadataLocationFromMetastore.equals(oldMetadataLocation)) {
+      throw new CommitFailedException(
+          "Base metadata location '%s' is not same as the current table metadata location '%s' for"
+              + " %s.%s",
+          oldMetadataLocation,
+          metadataLocationFromMetastore,
+          tableName.getDatabase(),
+          tableName.getTable());
+    }
+
+    try {
+      // Updating a BLMS table with etag. The BLMS server transactionally (1) checks that the etag
+      // of a table on server is the same as the etag provided by the client, and (2) updates the
+      // table (and its etag). The server returns an error containing message "etag mismatch", if
+      // the etag on server has changed.
+      client.updateTableParameters(
+          tableName, buildTableParameters(newMetadataLocation, metadata), etag);
+    } catch (AbortedException e) {
+      if (e.getMessage().toLowerCase().contains("etag mismatch")) {
+        throw new CommitFailedException(
+            "Updating table failed due to conflict updates (etag mismatch)");
+      }
+    }
+  }
+
+  private Table makeNewTable(TableMetadata metadata, String metadataFileLocation) {
+    Table.Builder builder = Table.newBuilder().setType(Table.Type.HIVE);
+    builder
+        .getHiveOptionsBuilder()
+        .setTableType("EXTERNAL_TABLE")
+        .setStorageDescriptor(
+            StorageDescriptor.newBuilder()
+                .setLocationUri(metadata.location())
+                .setInputFormat("org.apache.hadoop.mapred.FileInputFormat")
+                .setOutputFormat("org.apache.hadoop.mapred.FileOutputFormat")

Review Comment:
   Do you want to mirror Hive properties so that Hive can read these tables when using the same metastore?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253623412


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergTableException;
+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.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final BigLakeClient client;
+  private final FileIO fileIO;
+  private final TableName tableName;
+
+  BigLakeTableOperations(BigLakeClient client, FileIO fileIO, TableName tableName) {
+    this.client = client;
+    this.fileIO = fileIO;
+    this.tableName = tableName;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    // Must default to null.
+    String metadataLocation = null;
+    try {
+      HiveTableOptions hiveOptions = client.getTable(tableName).getHiveOptions();
+      if (!hiveOptions.containsParameters(METADATA_LOCATION_PROP)) {
+        throw new NoSuchIcebergTableException(
+            "Table %s is not a valid Iceberg table, metadata location not found", tableName());
+      }
+
+      metadataLocation = hiveOptions.getParametersOrThrow(METADATA_LOCATION_PROP);
+    } catch (NoSuchTableException e) {
+      if (currentMetadataLocation() != null) {
+        // Re-throws the exception because the table must exist in this case.

Review Comment:
   This is handled in `refresh` that calls this method. I don't think you need it here.
   
   The Hive catalog has as similar catch, but it is to translate the exception to `NoSuchTableException`.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253619727


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {

Review Comment:
   I don't think this is needed. If the database can't be loaded, then this should throw `NoSuchNamespaceException`.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253604158


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));

Review Comment:
   NoSuchNamespaceException.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1192701270


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeTableOperationsTest {

Review Comment:
   Done. BigLakeCatalogTest now inherits CatalogTests. I kept some additional tests in BigLakeCatalogTest and BigLakeTableOperationsTest, for what CatalogTests doesn't cover. 



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1186927752


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperationsTest.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import io.grpc.Status.Code;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeTableOperationsTest {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+  private static final String DB_ID = "db";
+  private static final String TABLE_ID = "tbl";
+  private static final TableName TABLE_NAME =
+      TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, DB_ID, TABLE_ID);
+  private static final TableIdentifier SPARK_TABLE_ID = TableIdentifier.of(DB_ID, TABLE_ID);
+
+  @Mock private BigLakeClient bigLakeClient;
+
+  private BigLakeCatalog bigLakeCatalog;
+  private String warehouseLocation;
+  private BigLakeTableOperations tableOps;
+
+  @Before
+  public void before() throws Exception {
+    this.bigLakeCatalog = new BigLakeCatalog();
+    this.warehouseLocation = tempFolder.newFolder("hive-warehouse").toString();
+
+    bigLakeCatalog.setConf(new Configuration());
+    bigLakeCatalog.initialize(
+        CATALOG_ID,
+        /* properties= */ ImmutableMap.of(
+            BigLakeCatalog.PROPERTIES_KEY_GCP_PROJECT,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation),
+        GCP_PROJECT,
+        GCP_REGION,
+        bigLakeClient);
+    this.tableOps = (BigLakeTableOperations) bigLakeCatalog.newTableOps(SPARK_TABLE_ID);
+  }
+
+  @Test
+  public void testDoFresh_fetchLatestMetadataFromBigLake() throws Exception {
+    Table createdTable = createTestTable();
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME)).thenReturn(createdTable);
+
+    tableOps.refresh();
+    assertEquals(
+        createdTable
+            .getHiveOptions()
+            .getParametersOrDefault(BigLakeTestUtils.METADATA_LOCATION_PROP, ""),
+        tableOps.currentMetadataLocation());
+
+    reset(bigLakeClient);
+    when(bigLakeClient.getTable(TABLE_NAME))
+        .thenThrow(new NoSuchTableException("error message getTable"));
+    // Refresh fails when table is not found but metadata already presents.
+    assertThrows(NoSuchTableException.class, () -> tableOps.refresh());
+  }
+
+  @Test
+  public void testDoFresh_failForNonIcebergTable() throws Exception {

Review Comment:
   Non-Iceberg tables should result in `NoSuchIcebergTableException`, not refresh failures.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1186926944


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+
+/** A client interface of Google BigLake service. */
+interface BigLakeClient {

Review Comment:
   What is the value of this interface? Do you expect someone to swap out the implementation for some reason? And if so, why not just swap out the entire catalog instead? That would make more sense to me. Then you wouldn't need a second interface that basically duplicates the public `Catalog` API.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1180623688


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final Configuration conf;

Review Comment:
   Sorry it is a leftover item of cleanup. Removed 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] coufon commented on pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1521051380

   > sorry for posting here, ( asking from a user perspective) does this catalog support reading and writing iceberg table using third party tools ? is it compatible with Iceberg REST catalog, can I use it with pyiceberg ?
   
   It provides the same functionalities that an Iceberg custom catalog supports (https://iceberg.apache.org/docs/latest/custom-catalog/), like the existing HiveCatalog. It supports read/write with Spark and Flink. It does not work with Trino (needs Trino integration here: https://github.com/trinodb/trino/tree/master/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog).
   
   The BigLake Metastore API is not the same as the Iceberg REST catalog API spec, but they should be compatible and convertible via a proxy. We are happy to explore how to make it work with Iceberg REST client. Please let me know if you have any use cases.
   
   BigLake Metastore works with PyIceberg, we have a Python client: https://cloud.google.com/python/docs/reference/biglake/latest. We need to contribute some code in PyIceberg for the integration.
   


-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268252433


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()),
+        name.getDatabase());
+  }
+
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getCatalog());
+  }
+
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table getTable(TableName name) {
+    if (name.getTable().isEmpty()) {
+      throw new NoSuchTableException("BigLake API does not allow tables with empty ID");
+    }
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", newName.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table deleteTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.deleteTable(
+                DeleteTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Iterable<Table> listTables(DatabaseName name) {
+    return convertException(
+        () ->
+            stub.listTables(ListTablesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getDatabase());
+  }
+
+  @Override
+  public void close() {
+    stub.close();
+  }
+
+  // Converts BigLake API errors to Iceberg errors.
+  private <T> T convertException(Supplier<T> result, String resourceId) {
+    try {
+      return result.get();
+    } catch (PermissionDeniedException e) {
+      throw new NotAuthorizedException(e, "BigLake API permission denied");
+    } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+      throw new AlreadyExistsException(e, "Namespace already exists: %s", resourceId);

Review Comment:
   How do you know that the call was for a namespace 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1268340175


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1272920511


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()),
+        name.getDatabase());
+  }
+
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getCatalog());
+  }
+
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table getTable(TableName name) {
+    if (name.getTable().isEmpty()) {
+      throw new NoSuchTableException("BigLake API does not allow tables with empty ID");
+    }
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", newName.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table deleteTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.deleteTable(
+                DeleteTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Iterable<Table> listTables(DatabaseName name) {
+    return convertException(
+        () ->
+            stub.listTables(ListTablesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getDatabase());
+  }
+
+  @Override
+  public void close() {
+    stub.close();
+  }
+
+  // Converts BigLake API errors to Iceberg errors.
+  private <T> T convertException(Supplier<T> result, String resourceId) {
+    try {
+      return result.get();
+    } catch (PermissionDeniedException e) {
+      throw new NotAuthorizedException(e, "BigLake API permission denied");
+    } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+      throw new AlreadyExistsException(e, "Namespace already exists: %s", resourceId);

Review Comment:
   It makes sense, I changed it to local try catch blocks.



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

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1272921002


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()),
+        name.getDatabase());
+  }
+
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getCatalog());
+  }
+
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table getTable(TableName name) {
+    if (name.getTable().isEmpty()) {
+      throw new NoSuchTableException("BigLake API does not allow tables with empty ID");
+    }
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", newName.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table deleteTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.deleteTable(
+                DeleteTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Iterable<Table> listTables(DatabaseName name) {
+    return convertException(
+        () ->
+            stub.listTables(ListTablesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getDatabase());
+  }
+
+  @Override
+  public void close() {
+    stub.close();
+  }
+
+  // Converts BigLake API errors to Iceberg errors.
+  private <T> T convertException(Supplier<T> result, String resourceId) {
+    try {
+      return result.get();
+    } catch (PermissionDeniedException e) {
+      throw new NotAuthorizedException(e, "BigLake API permission denied");

Review Comment:
   Sounds good. Removed "BigLake API".



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253628405


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,410 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.api.gax.rpc.InvalidArgumentException;
+import com.google.api.pathtemplate.ValidationException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            BigLakeCatalog.PROPERTIES_KEY_GCP_PROJECT,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    mockBigLakeClient = mock(BigLakeClient.class);
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() {
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testNamespaceWithSlash() {

Review Comment:
   This should be unnecessary because of the `supportsNamesWithSlashes` above. If you want to test BigLake specific behavior, create a new test case instead.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253605571


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();
+    } else if (namespace.levels().length == 1) {
+      return getMetadata(getDatabase(namespace));
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogPulginName;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return catalogProperties == null ? ImmutableMap.of() : catalogProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  private String getDatabaseLocation(String dbId) {
+    String warehouseLocation =
+        LocationUtil.stripTrailingSlash(
+            catalogProperties.get(CatalogProperties.WAREHOUSE_LOCATION));
+    Preconditions.checkNotNull(warehouseLocation, "Data warehouse location is not set");
+    return String.format("%s/%s.db", LocationUtil.stripTrailingSlash(warehouseLocation), dbId);
+  }
+
+  private static TableIdentifier getTableIdentifier(Table table) {

Review Comment:
   All of these methods should fix the `get` problem.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253598773


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();
+    } else if (namespace.levels().length == 1) {
+      return getMetadata(getDatabase(namespace));
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogPulginName;
+  }
+
+  @Override
+  protected Map<String, String> properties() {
+    return catalogProperties == null ? ImmutableMap.of() : catalogProperties;
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  private String getDatabaseLocation(String dbId) {

Review Comment:
   Iceberg methods do not typically use `get` because it is unnecessary in most cases. Either replace it with a more specific verb or omit 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261505415


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.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.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              projectId,
+              region);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.catalogPulginName = inputName;
+    this.properties = ImmutableMap.copyOf(properties);
+    this.projectId = projectId;
+    this.region = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.client = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = this.properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // Return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, io, tableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client, io, tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(databaseName(namespace)))
+        .map(BigLakeCatalog::tableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): %s", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    HiveDatabaseOptions.Builder optionsBuilder =
+        loadDatabase(namespace).toBuilder().getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(databaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261504419


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.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.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              projectId,
+              region);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.catalogPulginName = inputName;
+    this.properties = ImmutableMap.copyOf(properties);
+    this.projectId = projectId;
+    this.region = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.client = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = this.properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // Return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, io, tableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client, io, tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(databaseName(namespace)))
+        .map(BigLakeCatalog::tableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261503271


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261780302


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {

Review Comment:
   Change it to bigLakeProperties. If fields and arguments are the same, it throws errors like "'properties' hides a field. [HiddenField]".



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1261831123


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();

Review Comment:
   It is for @Mock decorator. I removed it and the rule.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260359905


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260218913


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260342807


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();
+    } else if (namespace.levels().length == 1) {
+      return getMetadata(getDatabase(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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1260356773


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);

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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253615290


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.

Review Comment:
   Can you point me to where this is happening? I don't think it is right for this to list databases. If tables can't be tracked directly in the empty namespace, then this should always return `ImmutableList.of()`.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253530482


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";

Review Comment:
   `location` is an overloaded term so we should avoid it. From the doc, it looks like this is either a region or a multi-region but the default value is a multi-region. Is the intent for this to be both?
   
   I think we should use `region` if possible. This should probably be required so there isn't a general `us` default.



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

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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253564549


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.

Review Comment:
   What does this mean by "otherwise catalog plugin will be used"? What is "catalog plugin" and how does that supply a catalog ID?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264750502


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.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.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {

Review Comment:
   How about a better name? Maybe `initName` to make it clear where this comes from?



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264755636


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);

Review Comment:
   My bad, region and project ID were used by constructing the BigLakeClient, but they are actually not used. I should move pulling region projectID to the other initialize method like catalog ID. 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264759136


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty, returns empty to unblock
+      // deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {

Review Comment:
   It is still needed. NoSuchNamespaceException may be thrown by BigLake client when the database or catalog is not found on the server side. 



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264760670


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));

Review Comment:
   Used isValidIdentifier instead, thanks!



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264606211


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              projectId,
+              region);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1266090458


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockServiceHelper mockServiceHelper;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(),
+            Arrays.<MockGrpcService>asList(new MockMetastoreService()));
+    mockServiceHelper.start();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.PROJECT_ID,
+            GCP_PROJECT,
+            GCPProperties.REGION,
+            GCP_REGION,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            temp.toAbsolutePath().toString());
+
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(mockServiceHelper.createChannelProvider())
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(CATALOG_ID, properties, new BigLakeClient(settings));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(CATALOG_ID, properties, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    if (bigLakeCatalogUsingMockService != null) {
+      bigLakeCatalogUsingMockService.close();
+    }
+
+    if (bigLakeCatalogUsingMockClient != null) {
+      bigLakeCatalogUsingMockClient.close();
+    }
+
+    if (mockServiceHelper != null) {
+      mockServiceHelper.stop();
+    }
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(temp.toAbsolutePath().toString() + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTableToDifferentDatabaseShouldFail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespaceShouldCreateCatalogWhenNamespaceIsEmpty() {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.empty(), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of("n0", "n1"), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenNamespaceIsNotEmpty() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.empty());
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.empty());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenNamespaceIsTooLong() {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of("n0", "n1"))).isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() {
+    assertThatThrownBy(
+            () -> bigLakeCatalogUsingMockClient.setProperties(Namespace.empty(), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class)
+        .hasMessage("Invalid BigLake database namespace: empty");
+
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of("db", "tbl"), ImmutableMap.of()))
+        .isInstanceOf(NoSuchNamespaceException.class)
+        .hasMessage("Invalid BigLake database namespace: db.tbl");
+  }
+
+  @Test
+  public void testSetPropertiesShouldSucceedForDatabase() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(
+                    HiveDatabaseOptions.newBuilder()
+                        .putParameters("key1", "value1")
+                        .putParameters("key2", "value2"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.setProperties(
+                Namespace.of("db"), ImmutableMap.of("key2", "value222", "key3", "value3")))
+        .isTrue();
+    verify(mockBigLakeClient, times(1))
+        .updateDatabaseParameters(
+            DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db"),
+            ImmutableMap.of("key1", "value1", "key2", "value222", "key3", "value3"));
+  }
+
+  @Test
+  public void testRemovePropertiesShouldFailWhenNamespacesAreInvalid() {

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] Prometheus1400 commented on pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "Prometheus1400 (via GitHub)" <gi...@apache.org>.
Prometheus1400 commented on PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#issuecomment-1707277313

   Great work, this feature is exactly what my team needs. Are there any updates?
   @coufon @rdblue 


-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335231916


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());

Review Comment:
   I find this a bit confusing. `validateDatabaseId` depends on the behavior of `databaseId`, but the namespace is passed in. Why not just use the same pattern as the other catalogs and call `isValidIdentifier` for validation before calling `databaseId`?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335234215


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        String dbId = databaseId(namespace);
+        validateDatabaseId(dbId, namespace);
+        client.deleteDatabase(databaseName(dbId));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Failed to drop namespace", e);

Review Comment:
   This log message is inaccurate. If the database didn't exist, then there was no failure. This method is idempotent.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335234418


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      client.deleteTable(tableName(dbId, identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDb = databaseId(from.namespace());
+    validateDatabaseId(fromDb, from.namespace());
+
+    String toDb = databaseId(to.namespace());
+    validateDatabaseId(toDb, to.namespace());
+
+    Preconditions.checkArgument(
+        fromDb.equals(toDb),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    client.renameTable(tableName(fromDb, from.name()), tableName(toDb, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, region, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Streams.stream(client.listDatabases(catalogName))
+          .map(BigLakeCatalog::namespace)
+          .collect(ImmutableList.toImmutableList());
+    }
+
+    // Database namespace does not have nested namespaces.
+    if (namespace.levels().length == 1) {
+      return ImmutableList.of();
+    }
+
+    throw new NoSuchNamespaceException("Invalid namespace: %s", namespace);
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        String dbId = databaseId(namespace);
+        validateDatabaseId(dbId, namespace);
+        client.deleteDatabase(databaseName(dbId));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);

Review Comment:
   This shouldn't warn, since the database doesn't exist. You should either throw an `IllegalArgumentException` to signal that it was an invalid name (not recommended) or return `false`.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1220695393


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,346 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  private BigLakeCatalog fakeBigLakeCatalog;
+
+  private BigLakeClient mockBigLakeClient;
+  private BigLakeCatalog mockBigLakeCatalog;
+
+  @BeforeEach
+  public void createCatalog() throws Exception {
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            BigLakeCatalog.PROPERTIES_KEY_GCP_PROJECT,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    BigLakeClient fakeBigLakeClient = new FakeBigLakeClient();
+    fakeBigLakeCatalog = new BigLakeCatalog();
+    fakeBigLakeCatalog.setConf(new Configuration());
+    fakeBigLakeCatalog.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, fakeBigLakeClient);
+
+    mockBigLakeClient = mock(BigLakeClient.class);
+    mockBigLakeCatalog = new BigLakeCatalog();
+    mockBigLakeCatalog.setConf(new Configuration());
+    mockBigLakeCatalog.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return fakeBigLakeCatalog;
+  }
+
+  // By pass this test from CatalogTests, because BigLake API does not support "/" in resource IDs.
+  @Test
+  public void testNamespaceWithSlash() {}

Review Comment:
   Done. Added the tests that validation of resource IDs fails.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1216946888


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,346 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.Rule;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @Rule public final MockitoRule mockito = MockitoJUnit.rule();
+  @TempDir public Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  private BigLakeCatalog fakeBigLakeCatalog;
+
+  private BigLakeClient mockBigLakeClient;
+  private BigLakeCatalog mockBigLakeCatalog;
+
+  @BeforeEach
+  public void createCatalog() throws Exception {
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            BigLakeCatalog.PROPERTIES_KEY_GCP_PROJECT,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    BigLakeClient fakeBigLakeClient = new FakeBigLakeClient();
+    fakeBigLakeCatalog = new BigLakeCatalog();
+    fakeBigLakeCatalog.setConf(new Configuration());
+    fakeBigLakeCatalog.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, fakeBigLakeClient);
+
+    mockBigLakeClient = mock(BigLakeClient.class);
+    mockBigLakeCatalog = new BigLakeCatalog();
+    mockBigLakeCatalog.setConf(new Configuration());
+    mockBigLakeCatalog.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return fakeBigLakeCatalog;
+  }
+
+  // By pass this test from CatalogTests, because BigLake API does not support "/" in resource IDs.
+  @Test
+  public void testNamespaceWithSlash() {}

Review Comment:
   Can you replace this with a test of the actual behavior? I'm assuming that you get a nice error message.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1180618458


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.ServiceFailureException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.Util;
+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.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable {
+
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "blms_endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "gcp_project";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "gcp_location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "blms_catalog";
+
+  public static final String HIVE_METASTORE_WAREHOUSE_DIR = "hive.metastore.warehouse.dir";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileSystem fs;
+  private FileIO fileIO;
+  private Configuration conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    if (!properties.containsKey(PROPERTIES_KEY_GCP_PROJECT)) {
+      throw new ValidationException("GCP project must be specified");
+    }
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+    BigLakeClient newClient;
+    try {
+      newClient =
+          new BigLakeClientImpl(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    if (this.conf == null) {
+      LOG.warn("No Hadoop Configuration was set, using the default environment Configuration");
+      this.conf = new Configuration();
+    }
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) {
+      this.conf.set(
+          HIVE_METASTORE_WAREHOUSE_DIR,
+          LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION)));
+    }
+
+    this.fs =
+        Util.getFs(
+            new Path(
+                LocationUtil.stripTrailingSlash(
+                    properties.get(CatalogProperties.WAREHOUSE_LOCATION))),
+            conf);
+
+    String fileIOImpl =
+        properties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.hadoop.HadoopFileIO");
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        conf,
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    client.deleteTable(
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    if (!fromDbId.equals(toDbId)) {
+      throw new ValidationException("New table name must be in the same database");
+    }
+
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {

Review Comment:
   Sorry for the confusion. There are two options: (1) link a <spark-catalog> to a physical <biglake-catalog>, so the full identifier of a table is just <spark-catalog>.<database/schema>.<table>. `CREATE/DROP <spark-catalog>` is supported: it creates/deletes <biglake-catalog> via API. (2) Use <spark-catalog>.<biglake-catalog>.<database/schema>.<table>.
   
   We choose (1) to avoid the long table identifier in (2). The limitation is that customers can't use two <biglake-catalog>s in the same <spark-catalog>, they have to install two <spark-catalog>s instead. We think it is OK, because <biglake-catalog> is usually an environmental level container (e.g., dev, staging, prod envs).
   
   My concern is, I am not sure whether (1) violates any design pattern of Spark namespaces. Please let me know if it does.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253603657


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::putParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "removeProperties is only supported for tables and databases, namespace {} is not"
+              + " supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();
+    properties.forEach(optionsBuilder::removeParameters);
+    client.updateDatabaseParameters(getDatabaseName(namespace), optionsBuilder.getParametersMap());
+    return true;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) {
+    if (namespace.levels().length == 0) {
+      // Calls getCatalog to check existence. BLMS catalog has no metadata today.
+      client.getCatalog(catalogName);
+      return new HashMap<String, String>();
+    } else if (namespace.levels().length == 1) {
+      return getMetadata(getDatabase(namespace));
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));

Review Comment:
   I think this should fail with `NoSuchNamespaceException`.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253600437


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".

Review Comment:
   This should be implemented.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253620119


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);
+      return false;
+    }
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = getDatabaseId(from.namespace());
+    String toDbId = getDatabaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId), "New table name must be in the same database");
+    client.renameTable(getTableName(fromDbId, from.name()), getTableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.levels().length == 0) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      client.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = namespace.level(0);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(getDatabaseLocation(dbId));
+
+      client.createDatabase(DatabaseName.of(projectId, location, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (namespace.levels().length != 0) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty (listTables is called as
+      // well), returns empty to unblock deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(client.listDatabases(catalogName))
+        .map(BigLakeCatalog::getNamespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.levels().length == 0) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        client.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        client.deleteDatabase(getDatabaseName(namespace));
+        // We don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // We can support database or catalog level config controlling file deletion in future.
+      } else {
+        throw new IllegalArgumentException(invalidNamespaceMessage(namespace));
+      }
+    } catch (NoSuchNamespaceException e) {
+      LOG.warn("Dropping namespace failed", e);
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    Database.Builder builder;
+    try {
+      builder = getDatabase(namespace).toBuilder();
+    } catch (IllegalArgumentException e) {
+      LOG.warn(
+          "setProperties is only supported for tables and databases, namespace {} is not supported",
+          namespace.levels().length == 0 ? "empty" : namespace.toString(),
+          e);
+      return false;
+    }
+
+    HiveDatabaseOptions.Builder optionsBuilder = builder.getHiveOptionsBuilder();

Review Comment:
   Is this updating or replacing the properties?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253608969


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(PROPERTIES_KEY_GCP_PROJECT), "GCP project must be specified");
+    String propProjectId = properties.get(PROPERTIES_KEY_GCP_PROJECT);
+    String propLocation =
+        properties.getOrDefault(PROPERTIES_KEY_GCP_LOCATION, DEFAULT_GCP_LOCATION);
+
+    BigLakeClient newClient;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      newClient =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  PROPERTIES_KEY_BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT),
+              propProjectId,
+              propLocation);
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+    initialize(inputName, properties, propProjectId, propLocation, newClient);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String propProjectId,
+      String propLocation,
+      BigLakeClient bigLakeClient) {
+    this.catalogPulginName = inputName;
+    this.catalogProperties = ImmutableMap.copyOf(properties);
+    this.projectId = propProjectId;
+    this.location = propLocation;
+    Preconditions.checkNotNull(bigLakeClient, "BigLake client must not be null");
+    this.client = bigLakeClient;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin will be used.
+    this.catalogId = properties.getOrDefault(PROPERTIES_KEY_BLMS_CATALOG, inputName);
+    this.catalogName = CatalogName.of(projectId, location, catalogId);
+    LOG.info("Use BigLake catalog: {}", catalogName.toString());
+
+    String fileIOImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    // The identifier of metadata tables is like "ns.table.files".
+    // We return a non-existing table in this case (empty table ID is disallowed in BigLake
+    // Metastore), loadTable will try loadMetadataTable.
+    if (identifier.namespace().levels().length > 1
+        && MetadataTableType.from(identifier.name()) != null) {
+      return new BigLakeTableOperations(
+          client, fileIO, getTableName(identifier.namespace().level(0), /* tableId= */ ""));
+    }
+
+    return new BigLakeTableOperations(
+        client,
+        fileIO,
+        getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = getDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? getDatabaseLocation(getDatabaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    // When deleting a BLMS catalog via `DROP NAMESPACE <catalog>`, this method is called for
+    // verifying catalog emptiness. `namespace` is empty in this case, we list databases in
+    // this catalog instead.
+    // TODO: to return all tables in all databases in a BLMS catalog instead of a "placeholder".
+    if (namespace.levels().length == 0) {
+      return Iterables.isEmpty(client.listDatabases(catalogName))
+          ? ImmutableList.of()
+          : ImmutableList.of(TableIdentifier.of("placeholder"));
+    }
+
+    return Streams.stream(client.listTables(getDatabaseName(namespace)))
+        .map(BigLakeCatalog::getTableIdentifier)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+    TableMetadata lastMetadata = ops.current();
+    try {
+      client.deleteTable(
+          getTableName(getDatabaseId(identifier.namespace()), /* tableId= */ identifier.name()));
+    } catch (NoSuchTableException e) {
+      LOG.warn("Dropping table failed", e);

Review Comment:
   This isn't a failure, it's supported by the API. Can you remove this log message?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253530482


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";

Review Comment:
   `location` is an overloaded term. Is this like a region?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1253558917


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements SupportsNamespaces, Configurable<Object> {
+
+  // TODO: to move the configs to GCPProperties.java.
+  // User provided properties.
+  // The endpoint of BigLake API. Optional, default to DEFAULT_BIGLAKE_SERVICE_ENDPOINT.
+  public static final String PROPERTIES_KEY_BIGLAKE_ENDPOINT = "biglake.endpoint";
+  // The GCP project ID. Required.
+  public static final String PROPERTIES_KEY_GCP_PROJECT = "biglake.project-id";
+  // The GCP location (https://cloud.google.com/bigquery/docs/locations). Optional, default to
+  // DEFAULT_GCP_LOCATION.
+  public static final String PROPERTIES_KEY_GCP_LOCATION = "biglake.location";
+  // The BLMS catalog ID. It is the container resource of databases and tables.
+  // It links a BLMS catalog with this Iceberg catalog.
+  public static final String PROPERTIES_KEY_BLMS_CATALOG = "biglake.catalog";
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+  public static final String DEFAULT_GCP_LOCATION = "us";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String catalogPulginName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private Object conf;
+  private String projectId;
+  private String location;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient client;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {

Review Comment:
   In general, argument and field names should match if they're the same thing.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335231388


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");

Review Comment:
   Nit: Should be "Failed to create BigLake client" to be more clear.



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335233680


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.

Review Comment:
   Why is this still TODO?



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1335233097


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin.
+  private String name;
+  private Map<String, String> properties;
+  private FileIO io;
+  private Object conf;
+
+  private BigLakeClient client;
+
+  private String projectId;
+  private String region;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private String warehouseLocation;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String initName, Map<String, String> initProperties) {
+    BigLakeClient initClient;
+    try {
+      // CatalogProperties.URI specifies the endpoint of BigLake API.
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      initClient =
+          new BigLakeClient(
+              initProperties.getOrDefault(CatalogProperties.URI, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(initName, initProperties, initClient);
+  }
+
+  @VisibleForTesting
+  void initialize(String initName, Map<String, String> initProperties, BigLakeClient initClient) {
+    this.name = initName;
+    this.properties = ImmutableMap.copyOf(initProperties);
+
+    Preconditions.checkArgument(initClient != null, "BigLake client must not be null");
+    this.client = initClient;
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.PROJECT_ID), "GCP project ID must be specified");
+    this.projectId = properties.get(GCPProperties.PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.REGION), "GCP region must be specified");
+    this.region = properties.get(GCPProperties.REGION);
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId = properties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, initName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    Preconditions.checkArgument(
+        properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION),
+        "Data warehouse location must be specified");
+    this.warehouseLocation =
+        LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+
+    String ioImpl =
+        properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(client);
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    return new BigLakeTableOperations(client, io, name(), tableName(dbId, identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String dbId = databaseId(identifier.namespace());
+    validateDatabaseId(dbId, identifier.namespace());
+
+    String locationUri = loadDatabase(dbId).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri) ? databaseLocation(dbId) : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames = ImmutableList.of();
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(client.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      String dbId = databaseId(namespace);
+      validateDatabaseId(dbId, namespace);
+      dbNames = ImmutableList.of(databaseName(dbId));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(client.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);

Review Comment:
   This can be much simpler:
   
   ```java
       return dbNames.stream()
           .flatMap(
               dbName ->
                   Streams.stream(client.listTables(dbName))
                       .map(BigLakeCatalog::tableIdentifier))
           .collect(ImmutableList.toImmutableList());
   ```



-- 
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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1179801693


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeTableOperations.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.AbortedException;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.SerDeInfo;
+import com.google.cloud.bigquery.biglake.v1.HiveTableOptions.StorageDescriptor;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Handles BigLake table operations. */
+public final class BigLakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeTableOperations.class);
+
+  private final Configuration conf;

Review Comment:
   We should avoid direct dependencies on Hadoop.  I don't see why this is necessary as it does not 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 #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1272528419


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeClient.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.api.gax.rpc.PermissionDeniedException;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.CreateCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.CreateTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.DeleteCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.DeleteTableRequest;
+import com.google.cloud.bigquery.biglake.v1.GetCatalogRequest;
+import com.google.cloud.bigquery.biglake.v1.GetDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.GetTableRequest;
+import com.google.cloud.bigquery.biglake.v1.ListDatabasesRequest;
+import com.google.cloud.bigquery.biglake.v1.ListTablesRequest;
+import com.google.cloud.bigquery.biglake.v1.LocationName;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceClient;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.RenameTableRequest;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import com.google.cloud.bigquery.biglake.v1.UpdateDatabaseRequest;
+import com.google.cloud.bigquery.biglake.v1.UpdateTableRequest;
+import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+
+/** A client of Google BigLake service. */
+final class BigLakeClient implements Closeable {
+
+  private final MetastoreServiceClient stub;
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param settings BigLake service settings
+   */
+  BigLakeClient(MetastoreServiceSettings settings) throws IOException {
+    this.stub = MetastoreServiceClient.create(settings);
+  }
+
+  /**
+   * Constructs a client of Google BigLake Service.
+   *
+   * @param biglakeEndpoint BigLake service gRPC endpoint, e.g., "biglake.googleapis.com:443"
+   */
+  BigLakeClient(String biglakeEndpoint) throws IOException {
+    this(MetastoreServiceSettings.newBuilder().setEndpoint(biglakeEndpoint).build());
+  }
+
+  public Catalog createCatalog(CatalogName name, Catalog catalog) {
+    return convertException(
+        () ->
+            stub.createCatalog(
+                CreateCatalogRequest.newBuilder()
+                    .setParent(LocationName.of(name.getProject(), name.getLocation()).toString())
+                    .setCatalogId(name.getCatalog())
+                    .setCatalog(catalog)
+                    .build()),
+        name.getCatalog());
+  }
+
+  public Catalog getCatalog(CatalogName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getCatalog(GetCatalogRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public void deleteCatalog(CatalogName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteCatalog(DeleteCatalogRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getCatalog());
+          }
+        },
+        name.getCatalog());
+  }
+
+  public Database createDatabase(DatabaseName name, Database db) {
+    return convertException(
+        () ->
+            stub.createDatabase(
+                CreateDatabaseRequest.newBuilder()
+                    .setParent(
+                        CatalogName.of(name.getProject(), name.getLocation(), name.getCatalog())
+                            .toString())
+                    .setDatabaseId(name.getDatabase())
+                    .setDatabase(db)
+                    .build()),
+        name.getDatabase());
+  }
+
+  public Database getDatabase(DatabaseName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.getDatabase(
+                GetDatabaseRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Database updateDatabaseParameters(DatabaseName name, Map<String, String> parameters) {
+    Database.Builder builder = Database.newBuilder().setName(name.toString());
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateDatabase(
+                UpdateDatabaseRequest.newBuilder()
+                    .setDatabase(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Iterable<Database> listDatabases(CatalogName name) {
+    return convertException(
+        () ->
+            stub.listDatabases(ListDatabasesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getCatalog());
+  }
+
+  public void deleteDatabase(DatabaseName name) {
+    convertException(
+        () -> {
+          try {
+            stub.deleteDatabase(
+                DeleteDatabaseRequest.newBuilder().setName(name.toString()).build());
+            return Empty.getDefaultInstance();
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchNamespaceException(
+                e, "Namespace does not exist: %s", name.getDatabase());
+          }
+        },
+        name.getDatabase());
+  }
+
+  public Table createTable(TableName name, Table table) {
+    return convertException(
+        () -> {
+          try {
+            return stub.createTable(
+                CreateTableRequest.newBuilder()
+                    .setParent(getDatabase(name).toString())
+                    .setTableId(name.getTable())
+                    .setTable(table)
+                    .build());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table getTable(TableName name) {
+    if (name.getTable().isEmpty()) {
+      throw new NoSuchTableException("BigLake API does not allow tables with empty ID");
+    }
+    return convertException(
+        () -> {
+          try {
+            return stub.getTable(GetTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table updateTableParameters(TableName name, Map<String, String> parameters, String etag) {
+    Table.Builder builder = Table.newBuilder().setName(name.toString()).setEtag(etag);
+    builder.getHiveOptionsBuilder().putAllParameters(parameters);
+    return convertException(
+        () -> {
+          try {
+            return stub.updateTable(
+                UpdateTableRequest.newBuilder()
+                    .setTable(builder)
+                    .setUpdateMask(FieldMask.newBuilder().addPaths("hive_options.parameters"))
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table renameTable(TableName name, TableName newName) {
+    return convertException(
+        () -> {
+          try {
+            return stub.renameTable(
+                RenameTableRequest.newBuilder()
+                    .setName(name.toString())
+                    .setNewName(newName.toString())
+                    .build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+            throw new AlreadyExistsException(e, "Table already exists: %s", newName.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Table deleteTable(TableName name) {
+    return convertException(
+        () -> {
+          try {
+            return stub.deleteTable(
+                DeleteTableRequest.newBuilder().setName(name.toString()).build());
+          } catch (PermissionDeniedException e) {
+            throw new NoSuchTableException(
+                e, "Table does not exist: %s (or permission denied)", name.getTable());
+          }
+        },
+        name.getTable());
+  }
+
+  public Iterable<Table> listTables(DatabaseName name) {
+    return convertException(
+        () ->
+            stub.listTables(ListTablesRequest.newBuilder().setParent(name.toString()).build())
+                .iterateAll(),
+        name.getDatabase());
+  }
+
+  @Override
+  public void close() {
+    stub.close();
+  }
+
+  // Converts BigLake API errors to Iceberg errors.
+  private <T> T convertException(Supplier<T> result, String resourceId) {
+    try {
+      return result.get();
+    } catch (PermissionDeniedException e) {
+      throw new NotAuthorizedException(e, "BigLake API permission denied");
+    } catch (com.google.api.gax.rpc.AlreadyExistsException e) {
+      throw new AlreadyExistsException(e, "Namespace already exists: %s", resourceId);

Review Comment:
   This seems extremely brittle to me. I think it will break in the future when this code is refactored or reused by someone that didn't know. I'd prefer not to have assumptions like this baked into the code paths.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264586467


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))
+        .isFalse();
+  }
+
+  @Test
+  public void testSetPropertiesShouldFailWhenNamespacesAreInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.setProperties(
+                    Namespace.of(new String[] {}), ImmutableMap.of()))

Review Comment:
   Done, cleaned up all Namespace.of.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264586503


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThat(bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {"n0", "n1"})))

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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264586713


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;

Review Comment:
   Thanks, 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264603102


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {

Review Comment:
   I changed it to a more informative test name. This catalog returns empty when list namespaces in a database or table, which 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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264603182


##########
gcp/src/test/java/org/apache/iceberg/gcp/biglake/BigLakeCatalogTest.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.testing.LocalChannelProvider;
+import com.google.api.gax.grpc.testing.MockGrpcService;
+import com.google.api.gax.grpc.testing.MockServiceHelper;
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.MetastoreServiceSettings;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.CatalogTests;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class BigLakeCatalogTest extends CatalogTests<BigLakeCatalog> {
+
+  @TempDir private Path temp;
+
+  private static final String GCP_PROJECT = "my-project";
+  private static final String GCP_REGION = "us";
+  private static final String CATALOG_ID = "biglake";
+
+  private String warehouseLocation;
+
+  // For tests using a BigLake catalog connecting to a mocked service.
+  private MockMetastoreService mockMetastoreService;
+  private MockServiceHelper mockServiceHelper;
+  private LocalChannelProvider channelProvider;
+  private BigLakeCatalog bigLakeCatalogUsingMockService;
+
+  // For tests using a BigLake catalog with a mocked client.
+  private BigLakeClient mockBigLakeClient = mock(BigLakeClient.class);;
+  private BigLakeCatalog bigLakeCatalogUsingMockClient;
+
+  @BeforeEach
+  public void before() throws Exception {
+    mockMetastoreService = new MockMetastoreService();
+    mockServiceHelper =
+        new MockServiceHelper(
+            UUID.randomUUID().toString(), Arrays.<MockGrpcService>asList(mockMetastoreService));
+    mockServiceHelper.start();
+
+    File warehouse = temp.toFile();
+    warehouseLocation = warehouse.getAbsolutePath();
+
+    ImmutableMap<String, String> properties =
+        ImmutableMap.of(
+            GCPProperties.BIGLAKE_PROJECT_ID,
+            GCP_PROJECT,
+            CatalogProperties.WAREHOUSE_LOCATION,
+            warehouseLocation);
+
+    channelProvider = mockServiceHelper.createChannelProvider();
+    MetastoreServiceSettings settings =
+        MetastoreServiceSettings.newBuilder()
+            .setTransportChannelProvider(channelProvider)
+            .setCredentialsProvider(NoCredentialsProvider.create())
+            .build();
+
+    bigLakeCatalogUsingMockService = new BigLakeCatalog();
+    bigLakeCatalogUsingMockService.setConf(new Configuration());
+    bigLakeCatalogUsingMockService.initialize(
+        CATALOG_ID,
+        properties,
+        GCP_PROJECT,
+        GCP_REGION,
+        new BigLakeClient(settings, GCP_PROJECT, GCP_REGION));
+
+    bigLakeCatalogUsingMockClient = new BigLakeCatalog();
+    bigLakeCatalogUsingMockClient.setConf(new Configuration());
+    bigLakeCatalogUsingMockClient.initialize(
+        CATALOG_ID, properties, GCP_PROJECT, GCP_REGION, mockBigLakeClient);
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    bigLakeCatalogUsingMockService.close();
+    bigLakeCatalogUsingMockClient.close();
+    mockServiceHelper.stop();
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  @Override
+  protected BigLakeCatalog catalog() {
+    return bigLakeCatalogUsingMockService;
+  }
+
+  @Override
+  protected boolean supportsNamesWithSlashes() {
+    return false;
+  }
+
+  @Test
+  public void testDefaultWarehouseWithDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder()
+                .setHiveOptions(HiveDatabaseOptions.newBuilder().setLocationUri("db_folder"))
+                .build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo("db_folder/table");
+  }
+
+  @Test
+  public void testDefaultWarehouseWithoutDatabaseLocation_asExpected() {
+    when(mockBigLakeClient.getDatabase(DatabaseName.of(GCP_PROJECT, "us", CATALOG_ID, "db")))
+        .thenReturn(
+            Database.newBuilder().setHiveOptions(HiveDatabaseOptions.getDefaultInstance()).build());
+
+    assertThat(
+            bigLakeCatalogUsingMockClient.defaultWarehouseLocation(
+                TableIdentifier.of("db", "table")))
+        .isEqualTo(warehouseLocation + "/db.db/table");
+  }
+
+  @Test
+  public void testRenameTable_differentDatabase_fail() {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.renameTable(
+                    TableIdentifier.of("db0", "t1"), TableIdentifier.of("db1", "t2")))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot rename table db0.t1 to db1.t2: database must match");
+  }
+
+  @Test
+  public void testCreateNamespace_createCatalogWhenEmptyNamespace() throws Exception {
+    bigLakeCatalogUsingMockClient.createNamespace(Namespace.of(new String[] {}), ImmutableMap.of());
+    verify(mockBigLakeClient, times(1))
+        .createCatalog(
+            CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID), Catalog.getDefaultInstance());
+  }
+
+  @Test
+  public void testCreateNamespaceShouldFailWhenInvalid() throws Exception {
+    assertThatThrownBy(
+            () ->
+                bigLakeCatalogUsingMockClient.createNamespace(
+                    Namespace.of(new String[] {"n0", "n1"}), ImmutableMap.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid namespace (too long): n0.n1");
+  }
+
+  @Test
+  public void testListNamespacesShouldReturnEmptyWhenInvalid() {
+    assertThat(bigLakeCatalogUsingMockClient.listNamespaces(Namespace.of("db"))).isEmpty();
+  }
+
+  @Test
+  public void testDropNamespaceShouldDeleteCatalogWhenEmptyNamespace() {
+    bigLakeCatalogUsingMockClient.dropNamespace(Namespace.of(new String[] {}));
+    verify(mockBigLakeClient, times(1))
+        .deleteCatalog(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID));
+  }
+
+  @Test
+  public void testListTablesShouldListTablesInAllDbsWhenNamespaceIsEmpty() {
+    DatabaseName db1Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1");
+    DatabaseName db2Name = DatabaseName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2");
+
+    TableName table1Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl1");
+    TableName table2Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db1", "tbl2");
+    TableName table3Name = TableName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID, "db2", "tbl3");
+
+    when(mockBigLakeClient.listDatabases(CatalogName.of(GCP_PROJECT, GCP_REGION, CATALOG_ID)))
+        .thenReturn(
+            ImmutableList.of(
+                Database.newBuilder().setName(db1Name.toString()).build(),
+                Database.newBuilder().setName(db2Name.toString()).build()));
+
+    when(mockBigLakeClient.listTables(db1Name))
+        .thenReturn(
+            ImmutableList.of(
+                Table.newBuilder().setName(table1Name.toString()).build(),
+                Table.newBuilder().setName(table2Name.toString()).build()));
+    when(mockBigLakeClient.listTables(db2Name))
+        .thenReturn(ImmutableList.of(Table.newBuilder().setName(table3Name.toString()).build()));
+
+    List<TableIdentifier> result = bigLakeCatalogUsingMockClient.listTables(Namespace.of());
+    assertThat(result)
+        .containsExactlyInAnyOrder(
+            TableIdentifier.of("db1", "tbl1"),
+            TableIdentifier.of("db1", "tbl2"),
+            TableIdentifier.of("db2", "tbl3"));
+  }
+
+  @Test
+  public void testDropNamespaceShouldFailWhenInvalid() throws Exception {

Review Comment:
   Changed invalid to a more descriptive 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] rdblue commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264755691


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.
+  private String name;
+  private Map<String, String> bigLakeProperties;
+  private FileIO io;
+  private Object conf;
+
+  private String bigLakeProjectId;
+  private String bigLakeRegion;
+  // BLMS catalog ID and fully qualified name.
+  private String catalogId;
+  private CatalogName catalogName;
+  private BigLakeClient bigLakeClient;
+
+  private CloseableGroup closeableGroup;
+
+  // Must have a no-arg constructor to be dynamically loaded
+  // initialize(String name, Map<String, String> properties) will be called to complete
+  // initialization
+  public BigLakeCatalog() {}
+
+  @Override
+  public void initialize(String inputName, Map<String, String> properties) {
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_PROJECT_ID),
+        "GCP project ID must be specified");
+    String projectId = properties.get(GCPProperties.BIGLAKE_PROJECT_ID);
+
+    Preconditions.checkArgument(
+        properties.containsKey(GCPProperties.BIGLAKE_GCP_REGION), "GCP region must be specified");
+    String region = properties.get(GCPProperties.BIGLAKE_GCP_REGION);
+
+    BigLakeClient client;
+    try {
+      // TODO: to add more auth options of the client. Currently it uses default auth
+      // (https://github.com/googleapis/google-cloud-java#application-default-credentials)
+      // that works on GCP services (e.g., GCE, GKE, Dataproc).
+      client =
+          new BigLakeClient(
+              properties.getOrDefault(
+                  GCPProperties.BIGLAKE_ENDPOINT, DEFAULT_BIGLAKE_SERVICE_ENDPOINT));
+    } catch (IOException e) {
+      throw new ServiceFailureException(e, "Creating BigLake client failed");
+    }
+
+    initialize(inputName, properties, projectId, region, client);
+  }
+
+  @VisibleForTesting
+  void initialize(
+      String inputName,
+      Map<String, String> properties,
+      String projectId,
+      String region,
+      BigLakeClient client) {
+    this.name = inputName;
+    this.bigLakeProperties = ImmutableMap.copyOf(properties);
+    this.bigLakeProjectId = projectId;
+    this.bigLakeRegion = region;
+    Preconditions.checkNotNull(client, "BigLake client must not be null");
+    this.bigLakeClient = client;
+
+    // Users can specify the BigLake catalog ID, otherwise catalog plugin name will be used.
+    // For example, "spark.sql.catalog.<plugin-name>=org.apache.iceberg.spark.SparkCatalog"
+    // specifies the plugin name "<plugin-name>".
+    this.catalogId =
+        this.bigLakeProperties.getOrDefault(GCPProperties.BIGLAKE_CATALOG_ID, inputName);
+    this.catalogName = CatalogName.of(projectId, region, catalogId);
+
+    String ioImpl =
+        this.bigLakeProperties.getOrDefault(
+            CatalogProperties.FILE_IO_IMPL, ResolvingFileIO.class.getName());
+    this.io = CatalogUtil.loadFileIO(ioImpl, this.bigLakeProperties, conf);
+
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(io);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier identifier) {
+    return new BigLakeTableOperations(
+        bigLakeClient,
+        io,
+        name(),
+        tableName(databaseId(identifier.namespace()), identifier.name()));
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier identifier) {
+    String locationUri = loadDatabase(identifier.namespace()).getHiveOptions().getLocationUri();
+    return String.format(
+        "%s/%s",
+        Strings.isNullOrEmpty(locationUri)
+            ? databaseLocation(databaseId(identifier.namespace()))
+            : locationUri,
+        identifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    ImmutableList<DatabaseName> dbNames;
+    if (namespace.isEmpty()) {
+      dbNames =
+          Streams.stream(bigLakeClient.listDatabases(catalogName))
+              .map(db -> DatabaseName.parse(db.getName()))
+              .collect(ImmutableList.toImmutableList());
+    } else {
+      dbNames = ImmutableList.of(databaseName(namespace));
+    }
+
+    ImmutableList.Builder<TableIdentifier> result = ImmutableList.builder();
+    dbNames.stream()
+        .map(
+            dbName ->
+                Streams.stream(bigLakeClient.listTables(dbName))
+                    .map(BigLakeCatalog::tableIdentifier)
+                    .collect(ImmutableList.toImmutableList()))
+        .forEach(result::addAll);
+    return result.build();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = null;
+    TableMetadata lastMetadata = null;
+    if (purge) {
+      ops = newTableOps(identifier);
+      // TODO: to catch NotFoundException as in https://github.com/apache/iceberg/pull/5510.
+      lastMetadata = ops.current();
+    }
+
+    try {
+      bigLakeClient.deleteTable(tableName(databaseId(identifier.namespace()), identifier.name()));
+    } catch (NoSuchTableException e) {
+      return false;
+    }
+
+    if (ops != null && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    String fromDbId = databaseId(from.namespace());
+    String toDbId = databaseId(to.namespace());
+
+    Preconditions.checkArgument(
+        fromDbId.equals(toDbId),
+        "Cannot rename table %s to %s: database must match",
+        from.toString(),
+        to.toString());
+    bigLakeClient.renameTable(tableName(fromDbId, from.name()), tableName(toDbId, to.name()));
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    if (namespace.isEmpty()) {
+      // Used by `CREATE NAMESPACE <catalog>`. Create a BLMS catalog linked with Iceberg catalog.
+      bigLakeClient.createCatalog(catalogName, Catalog.getDefaultInstance());
+      LOG.info("Created BigLake catalog: {}", catalogName.toString());
+    } else if (namespace.levels().length == 1) {
+      // Create a database.
+      String dbId = databaseId(namespace);
+      Database.Builder builder = Database.newBuilder().setType(Database.Type.HIVE);
+      builder
+          .getHiveOptionsBuilder()
+          .putAllParameters(metadata)
+          .setLocationUri(databaseLocation(dbId));
+
+      bigLakeClient.createDatabase(
+          DatabaseName.of(bigLakeProjectId, bigLakeRegion, catalogId, dbId), builder.build());
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Invalid namespace (too long): %s", namespace));
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    if (!namespace.isEmpty()) {
+      // BLMS does not support namespaces under database or tables, returns empty.
+      // It is called when dropping a namespace to make sure it's empty, returns empty to unblock
+      // deletion.
+      return ImmutableList.of();
+    }
+
+    return Streams.stream(bigLakeClient.listDatabases(catalogName))
+        .map(BigLakeCatalog::namespace)
+        .collect(ImmutableList.toImmutableList());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    try {
+      if (namespace.isEmpty()) {
+        // Used by `DROP NAMESPACE <catalog>`. Deletes the BLMS catalog linked by Iceberg catalog.
+        bigLakeClient.deleteCatalog(catalogName);
+        LOG.info("Deleted BigLake catalog: {}", catalogName.toString());
+      } else if (namespace.levels().length == 1) {
+        bigLakeClient.deleteDatabase(databaseName(namespace));
+        // Don't delete the data file folder for safety. It aligns with HMS's default behavior.
+        // To support database or catalog level config controlling file deletion in future.
+      } else {
+        LOG.warn("Invalid namespace (too long): {}", namespace);
+        return false;
+      }
+    } catch (NoSuchNamespaceException e) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {

Review Comment:
   I think this should handle `location` separately because that's a property that's stored in the `Database` object.



-- 
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] coufon commented on a diff in pull request #7412: GCP: Add Iceberg Catalog for GCP BigLake Metastore

Posted by "coufon (via GitHub)" <gi...@apache.org>.
coufon commented on code in PR #7412:
URL: https://github.com/apache/iceberg/pull/7412#discussion_r1264753673


##########
gcp/src/main/java/org/apache/iceberg/gcp/biglake/BigLakeCatalog.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.gcp.biglake;
+
+import com.google.cloud.bigquery.biglake.v1.Catalog;
+import com.google.cloud.bigquery.biglake.v1.CatalogName;
+import com.google.cloud.bigquery.biglake.v1.Database;
+import com.google.cloud.bigquery.biglake.v1.DatabaseName;
+import com.google.cloud.bigquery.biglake.v1.HiveDatabaseOptions;
+import com.google.cloud.bigquery.biglake.v1.Table;
+import com.google.cloud.bigquery.biglake.v1.TableName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+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.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.ServiceFailureException;
+import org.apache.iceberg.gcp.GCPProperties;
+import org.apache.iceberg.hadoop.Configurable;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.ResolvingFileIO;
+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.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.LocationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Iceberg BigLake Metastore (BLMS) Catalog implementation. */
+public final class BigLakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+
+  public static final String DEFAULT_BIGLAKE_SERVICE_ENDPOINT = "biglake.googleapis.com:443";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BigLakeCatalog.class);
+
+  // The name of this Iceberg catalog plugin: spark.sql.catalog.<catalog_plugin>.

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