This is an automated email from the ASF dual-hosted git repository.

danny0405 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new d45df4dd684 [HUDI-6231] Handle glue comments for meta sync (#8740)
d45df4dd684 is described below

commit d45df4dd6840ea045b38ec0b30e5ceab147a4456
Author: Nicolas Paris <[email protected]>
AuthorDate: Wed Jun 14 03:29:49 2023 +0200

    [HUDI-6231] Handle glue comments for meta sync (#8740)
    
    * Handle comment has been modified or not
    
    ---------
    
    Co-authored-by: David Ayache <[email protected]>
---
 .../hudi/aws/sync/AWSGlueCatalogSyncClient.java    | 70 ++++++++++++++++++++++
 1 file changed, 70 insertions(+)

diff --git 
a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java 
b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java
index cf3a6d2dcf0..77b2f26956b 100644
--- 
a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java
+++ 
b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java
@@ -19,11 +19,13 @@
 package org.apache.hudi.aws.sync;
 
 import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.TableSchemaResolver;
 import org.apache.hudi.common.util.CollectionUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.GlueCatalogSyncClientConfig;
 import org.apache.hudi.hive.HiveSyncConfig;
 import org.apache.hudi.sync.common.HoodieSyncClient;
+import org.apache.hudi.sync.common.model.FieldSchema;
 import org.apache.hudi.sync.common.model.Partition;
 
 import com.amazonaws.services.glue.AWSGlue;
@@ -237,6 +239,74 @@ public class AWSGlueCatalogSyncClient extends 
HoodieSyncClient {
     }
   }
 
+  private void setComments(List<Column> columns, Map<String, Option<String>> 
commentsMap) {
+    columns.forEach(column -> {
+      String comment = commentsMap.getOrDefault(column.getName(), 
Option.empty()).orElse(null);
+      column.setComment(comment);
+    });
+  }
+
+  private String getTableDoc() {
+    try {
+      return new 
TableSchemaResolver(metaClient).getTableAvroSchema(true).getDoc();
+    } catch (Exception e) {
+      throw new HoodieGlueSyncException("Failed to get schema's doc from 
storage : ", e);
+    }
+  }
+
+  @Override
+  public List<FieldSchema> getStorageFieldSchemas() {
+    try {
+      return new TableSchemaResolver(metaClient).getTableAvroSchema(true)
+          .getFields()
+          .stream()
+          .map(f -> new FieldSchema(f.name(), f.schema().getType().getName(), 
f.doc()))
+          .collect(Collectors.toList());
+    } catch (Exception e) {
+      throw new HoodieGlueSyncException("Failed to get field schemas from 
storage : ", e);
+    }
+  }
+
+  @Override
+  public boolean updateTableComments(String tableName, List<FieldSchema> 
fromMetastore, List<FieldSchema> fromStorage) {
+    Table table = getTable(awsGlue, databaseName, tableName);
+
+    Map<String, Option<String>> commentsMap = 
fromStorage.stream().collect(Collectors.toMap(FieldSchema::getName, 
FieldSchema::getComment));
+
+    StorageDescriptor storageDescriptor = table.getStorageDescriptor();
+    List<Column> columns = storageDescriptor.getColumns();
+    setComments(columns, commentsMap);
+
+    List<Column> partitionKeys = table.getPartitionKeys();
+    setComments(partitionKeys, commentsMap);
+
+    String tableDescription = getTableDoc();
+
+    if (getTable(awsGlue, databaseName, 
tableName).getStorageDescriptor().equals(storageDescriptor)
+        && getTable(awsGlue, databaseName, 
tableName).getPartitionKeys().equals(partitionKeys)) {
+      // no comments have been modified / added
+      return false;
+    } else {
+      final Date now = new Date();
+      TableInput updatedTableInput = new TableInput()
+          .withName(tableName)
+          .withDescription(tableDescription)
+          .withTableType(table.getTableType())
+          .withParameters(table.getParameters())
+          .withPartitionKeys(partitionKeys)
+          .withStorageDescriptor(storageDescriptor)
+          .withLastAccessTime(now)
+          .withLastAnalyzedTime(now);
+
+      UpdateTableRequest request = new UpdateTableRequest()
+          .withDatabaseName(databaseName)
+          .withTableInput(updatedTableInput);
+
+      awsGlue.updateTable(request);
+      return true;
+    }
+  }
+
   @Override
   public void updateTableSchema(String tableName, MessageType newSchema) {
     // ToDo Cascade is set in Hive meta sync, but need to investigate how to 
configure it for Glue meta

Reply via email to