jackye1995 commented on a change in pull request #1633:
URL: https://github.com/apache/iceberg/pull/1633#discussion_r513693893



##########
File path: aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.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.aws.glue;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.aws.IcebergAWSConfigs;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.awscore.exception.AwsServiceException;
+import software.amazon.awssdk.services.glue.GlueClient;
+import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.CreateTableRequest;
+import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.DeleteTableRequest;
+import software.amazon.awssdk.services.glue.model.EntityNotFoundException;
+import software.amazon.awssdk.services.glue.model.GetDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.GetDatabaseResponse;
+import software.amazon.awssdk.services.glue.model.GetDatabasesRequest;
+import software.amazon.awssdk.services.glue.model.GetDatabasesResponse;
+import software.amazon.awssdk.services.glue.model.GetTableRequest;
+import software.amazon.awssdk.services.glue.model.GetTableResponse;
+import software.amazon.awssdk.services.glue.model.GetTablesRequest;
+import software.amazon.awssdk.services.glue.model.GetTablesResponse;
+import software.amazon.awssdk.services.glue.model.InvalidInputException;
+import software.amazon.awssdk.services.glue.model.Table;
+import software.amazon.awssdk.services.glue.model.TableInput;
+import software.amazon.awssdk.services.glue.model.UpdateDatabaseRequest;
+
+public class GlueCatalog extends BaseMetastoreCatalog implements Closeable, 
SupportsNamespaces {
+
+  private static final Logger LOG = LoggerFactory.getLogger(GlueCatalog.class);
+
+  private final Configuration conf;
+  private final GlueClient glue;
+  private final String catalogId;
+  private final String catalogName;
+  private final String warehousePath;
+  private final boolean skipArchive;
+
+  /**
+   * Constructor that initializes all args from Hadoop configuration, used by 
engines like Spark and Flink
+   * @param conf Hadoop config passed from engines
+   */
+  public GlueCatalog(Configuration conf) {
+    this(
+        conf,
+        conf.get(IcebergAWSConfigs.ICEBERG_AWS_GLUE_CATALOG_ID),
+        conf.get(IcebergAWSConfigs.ICEBERG_AWS_GLUE_WAREHOUSE_PATH),
+        conf.getBoolean(IcebergAWSConfigs.ICEBERG_AWS_GLUE_SKIP_ARCHIVE,
+            IcebergAWSConfigs.ICEBERG_AWS_GLUE_SKIP_ARCHIVE_DEFAULT)
+    );
+  }
+
+  /**
+   * Constructor with default glue client
+   * @param conf Hadoop config, passed in for users of HadoopIO
+   * @param catalogId Glue catalog ID, which is the AWS account ID. When null, 
it uses the account of the Glue client.
+   * @param warehousePath warehouse path, such as 
s3://my-iceberg-bucket/metadata_prod
+   * @param skipArchive if Glue should skip an old table version when creating 
a new version in a commit.
+   *                    By default Glue archives all old table versions after 
an UpdateTable call.
+   *                    but Glue has a default max number of archived table 
versions (can be increased).
+   *                    So for streaming use case with lots of commits, it is 
recommended to turn this feature off.
+   */
+  public GlueCatalog(
+      Configuration conf,
+      String catalogId,
+      String warehousePath,
+      boolean skipArchive) {
+    this(conf, GlueClient.create(), catalogId, warehousePath, skipArchive);
+  }
+
+  @Override
+  public String name() {
+    return catalogName;
+  }
+
+  /**
+   * All-arg constructor
+   * @param conf Hadoop config, passed in for users of HadoopIO
+   * @param glue Glue client
+   * @param catalogId Glue catalog ID, which is the AWS account ID. When null, 
it uses the account of the Glue client.
+   * @param warehousePath warehouse path, such as 
s3://my-iceberg-bucket/metadata_prod
+   * @param skipArchive if Glue should skip an old table version when creating 
a new version in a commit.
+   *                    By default Glue archives all old table versions after 
an UpdateTable call.
+   *                    but Glue has a default max number of archived table 
versions (can be increased).
+   *                    So for streaming use case with lots of commits, it is 
recommended to turn this feature off.
+   */
+  public GlueCatalog(
+      Configuration conf,
+      GlueClient glue,
+      String catalogId,
+      String warehousePath,
+      boolean skipArchive) {
+    this.conf = Preconditions.checkNotNull(conf, "Cannot initialize 
GlueCatalog because Hadoop config is null");
+    this.glue = Preconditions.checkNotNull(glue, "Cannot initialize 
GlueCatalog because GlueClient is null");
+    this.catalogId = catalogId;
+    this.catalogName = "glue." + catalogId; // for logging purpose
+    this.warehousePath = warehousePath;
+    this.skipArchive = skipArchive;
+    validateWarehousePath();
+  }
+
+  private void validateWarehousePath() {
+    Preconditions.checkArgument(warehousePath != null && 
warehousePath.length() > 0,
+        "Cannot initialize GlueCatalog because warehousePath must not be 
null");
+    Preconditions.checkArgument(warehousePath.charAt(warehousePath.length() - 
1) != '/',
+        "Cannot initialize GlueCatalog because warehousePath %s must not end 
with /", warehousePath);
+  }
+
+  @Override
+  public void close() throws IOException {
+    glue.close();
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new GlueTableOperations(conf, glue, catalogId, skipArchive, 
tableIdentifier);
+  }
+
+  /**
+   * This method produces the same result as using a HiveCatalog.
+   * If databaseUri exists for the Glue database URI, the default location is 
databaseUri/tableName.
+   * If not, the default location is warehousePath/databaseName.db/tableName
+   * @param tableIdentifier table id
+   * @return default warehouse path
+   */
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) {
+    // check if value is set in database
+    GetDatabaseResponse response = 
glue.getDatabase(GetDatabaseRequest.builder()
+        .name(IcebergToGlueConverter.getDatabaseName(tableIdentifier))
+        .build());
+    String dbLocationUri = response.database().locationUri();
+    if (dbLocationUri != null) {
+      return String.format("%s/%s", dbLocationUri, tableIdentifier.name());
+    }
+    return String.format(
+        "%s/%s.db/%s",
+        warehousePath,
+        IcebergToGlueConverter.getDatabaseName(tableIdentifier),
+        tableIdentifier.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    namespaceExists(namespace);
+    // should be safe to list all before returning the list, instead of 
dynamically load the list.
+    String nextToken = null;
+    List<TableIdentifier> results = new ArrayList<>();
+    do {
+      GetTablesResponse response = glue.getTables(GetTablesRequest.builder()
+          .catalogId(catalogId)
+          .databaseName(IcebergToGlueConverter.toDatabaseName(namespace))
+          .nextToken(nextToken)
+          .build());
+      if (response.hasTableList()) {
+        results.addAll(response.tableList().stream()
+            .map(GlueToIcebergConverter::toTableId)
+            .collect(Collectors.toList()));
+        nextToken = response.nextToken();
+      }
+    } while (nextToken != null);
+    LOG.debug("Listing of namespace: {} resulted in the following tables: {}", 
namespace, results);
+    return results;
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    try {
+      TableOperations ops = newTableOps(identifier);
+      TableMetadata lastMetadata = ops.current();
+      glue.deleteTable(DeleteTableRequest.builder()
+          .catalogId(catalogId)
+          .databaseName(IcebergToGlueConverter.getDatabaseName(identifier))
+          .name(identifier.name())
+          .build());
+      LOG.info("Successfully dropped table {} from Glue", identifier);
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        LOG.info("Glue table {} data purged", identifier);
+      }
+      LOG.info("Dropped table: {}", identifier);
+      return true;
+    } catch (EntityNotFoundException e) {
+      LOG.error("Cannot drop table {} because table not found or not 
accessible", identifier, e);
+      return false;
+    } catch (Exception e) {
+      LOG.error("Cannot complete drop table operation for {} due to unexpected 
exception", identifier, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Rename table in Glue is a drop table and create table.
+   * @param from identifier of the table to rename
+   * @param to new table name
+   */
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    // check new namespace exists
+    if (!namespaceExists(to.namespace())) {
+      throw new NoSuchNamespaceException("Cannot rename %s to %s because 
namespace %s does not exist",
+          from, to, to.namespace());
+    }
+    // keep metadata
+    Table fromTable = null;
+    String fromTableDbName = IcebergToGlueConverter.getDatabaseName(from);
+    String fromTableName = IcebergToGlueConverter.getTableName(from);
+    try {
+      GetTableResponse response = glue.getTable(GetTableRequest.builder()
+          .catalogId(catalogId)
+          .databaseName(fromTableDbName)
+          .name(fromTableName)
+          .build());
+      fromTable = response.table();
+    } catch (EntityNotFoundException e) {
+      throw new NoSuchTableException(e, "Cannot rename %s because the table 
does not exist in Glue", from);
+    }
+    dropTable(from, false);
+    // use the same Glue info to create the new table, pointing to the old 
metadata
+    TableInput.Builder tableInputBuilder = TableInput.builder()
+        .owner(fromTable.owner())
+        .tableType(fromTable.tableType())
+        .parameters(fromTable.parameters());
+    try {
+      glue.createTable(CreateTableRequest.builder()
+          .catalogId(catalogId)
+          .databaseName(IcebergToGlueConverter.getDatabaseName(to))
+          
.tableInput(tableInputBuilder.name(IcebergToGlueConverter.getTableName(to)).build())
+          .build());
+      LOG.info("Renamed table from {} to {}", from, to);
+    } catch (AwsServiceException | NoSuchTableException e) {
+      // rollback, recreate old table and rethrow exception
+      LOG.error("Fail to rename table from {} to {}, rollback the old table", 
from, to, e);
+      glue.createTable(CreateTableRequest.builder()
+          .catalogId(catalogId)
+          .databaseName(fromTableDbName)
+          .tableInput(tableInputBuilder.name(fromTableName).build())
+          .build());
+      throw e;
+    }
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> 
metadata) {
+    try {
+      glue.createDatabase(CreateDatabaseRequest.builder()
+          .catalogId(catalogId)
+          .databaseInput(IcebergToGlueConverter.toDatabaseInput(namespace, 
metadata))
+          .build());
+      LOG.info("Created namespace: {}", namespace);
+    } catch (software.amazon.awssdk.services.glue.model.AlreadyExistsException 
e) {
+      throw new AlreadyExistsException("Cannot create namespace %s because it 
already exists in Glue", namespace);
+    }
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws 
NoSuchNamespaceException {
+    if (!namespace.isEmpty()) {
+      // if it is not a list all op, just check if the namespace exists and 
return itself.
+      if (namespaceExists(namespace)) {
+        return Lists.newArrayList(namespace);
+      }
+      throw new NoSuchNamespaceException(
+          "Glue does not support nested namespace, cannot list namespaces 
under %s", namespace);
+    }
+    // should be safe to list all before returning the list, instead of 
dynamically load the list.
+    String nextToken = null;
+    List<Namespace> results = new ArrayList<>();
+    do {
+      GetDatabasesResponse response = 
glue.getDatabases(GetDatabasesRequest.builder()
+          .catalogId(catalogId)
+          .nextToken(nextToken)
+          .build());
+      if (response.hasDatabaseList()) {
+        results.addAll(response.databaseList().stream()
+            .map(GlueToIcebergConverter::toNamespace)
+            .collect(Collectors.toList()));
+        nextToken = response.nextToken();
+      }
+    } while (nextToken != null);
+    LOG.debug("Listing namespace {} returned namespaces: {}", namespace, 
results);
+    return results;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws 
NoSuchNamespaceException {
+    String databaseName = IcebergToGlueConverter.toDatabaseName(namespace);
+    try {
+      GetDatabaseResponse response = 
glue.getDatabase(GetDatabaseRequest.builder()
+          .catalogId(catalogId)
+          .name(databaseName)
+          .build());
+      Map<String, String> result = response.database().parameters();
+      LOG.debug("Loaded metadata for namespace {} found {}", namespace, 
result);
+      return result;
+    } catch (InvalidInputException e) {
+      throw new NoSuchNamespaceException("invalid input for namespace %s, 
error message: %s",
+          namespace, e.getMessage());
+    } catch (EntityNotFoundException e) {
+      throw new NoSuchNamespaceException("fail to find Glue database for 
namespace %s, error message: %s",
+          databaseName, e.getMessage());
+    }
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws 
NamespaceNotEmptyException {
+    namespaceExists(namespace);
+    List<TableIdentifier> tableIdentifiers = listTables(namespace);
+    if (!CollectionUtils.isEmpty(tableIdentifiers)) {
+      throw new NamespaceNotEmptyException("Cannot drop namespace %s because 
it is not empty. " +
+          "The following tables still exist under the namespace: %s", 
namespace, tableIdentifiers);
+    }
+    glue.deleteDatabase(DeleteDatabaseRequest.builder()
+        .catalogId(catalogId)
+        .name(IcebergToGlueConverter.toDatabaseName(namespace))
+        .build());
+    LOG.info("Dropped namespace: {}", namespace);
+    // Always successful, otherwise exception is thrown
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> 
properties) throws NoSuchNamespaceException {
+    Map<String, String> parameter = Maps.newHashMap();

Review comment:
       Sure, `parameter` is used to mean `properties` in Glue that's why I used 
it. Will update to 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.

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