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

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


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

Reply via email to