nsivabalan commented on a change in pull request #5076:
URL: https://github.com/apache/hudi/pull/5076#discussion_r831580497



##########
File path: 
hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/AbstractHiveSyncHoodieClient.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.hive;
+
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.TableSchemaResolver;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+import org.apache.hudi.sync.common.HoodieSyncException;
+import org.apache.hudi.sync.common.model.Partition;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.parquet.schema.MessageType;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Base class to sync Hudi tables with Hive based metastores, such as Hive 
server, HMS or managed Hive services.
+ */
+public abstract class AbstractHiveSyncHoodieClient extends 
AbstractSyncHoodieClient {
+
+  protected final HoodieTimeline activeTimeline;
+  protected final HiveSyncConfig syncConfig;
+  protected final Configuration hadoopConf;
+  protected final PartitionValueExtractor partitionValueExtractor;
+
+  public AbstractHiveSyncHoodieClient(HiveSyncConfig syncConfig, Configuration 
hadoopConf, FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, 
syncConfig.useFileListingFromMetadata, syncConfig.withOperationField, fs);
+    this.syncConfig = syncConfig;
+    this.hadoopConf = hadoopConf;
+    this.partitionValueExtractor = 
ReflectionUtils.loadClass(syncConfig.partitionValueExtractorClass);
+    this.activeTimeline = 
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
+  }
+
+  public HoodieTimeline getActiveTimeline() {
+    return activeTimeline;
+  }
+
+  /**
+   * Iterate over the storage partitions and find if there are any new 
partitions that need to be added or updated.
+   * Generate a list of PartitionEvent based on the changes required.
+   */
+  protected List<PartitionEvent> getPartitionEvents(List<Partition> 
tablePartitions, List<String> partitionStoragePartitions, boolean 
isDropPartition) {
+    Map<String, String> paths = new HashMap<>();
+    for (Partition tablePartition : tablePartitions) {
+      List<String> hivePartitionValues = tablePartition.getValues();
+      String fullTablePartitionPath =
+          Path.getPathWithoutSchemeAndAuthority(new 
Path(tablePartition.getStorageLocation())).toUri().getPath();
+      paths.put(String.join(", ", hivePartitionValues), 
fullTablePartitionPath);
+    }
+
+    List<PartitionEvent> events = new ArrayList<>();
+    for (String storagePartition : partitionStoragePartitions) {
+      Path storagePartitionPath = 
FSUtils.getPartitionPath(syncConfig.basePath, storagePartition);
+      String fullStoragePartitionPath = 
Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath();
+      // Check if the partition values or if hdfs path is the same
+      List<String> storagePartitionValues = 
partitionValueExtractor.extractPartitionValuesInPath(storagePartition);
+
+      if (isDropPartition) {
+        events.add(PartitionEvent.newPartitionDropEvent(storagePartition));
+      } else {
+        if (!storagePartitionValues.isEmpty()) {

Review comment:
       is storagePartitionValues just the partition name as string or full 
partition path? wondering what happens incase of non - partitioned partition 
path?

##########
File path: 
hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/AbstractHiveSyncHoodieClient.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.hive;
+
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.TableSchemaResolver;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+import org.apache.hudi.sync.common.HoodieSyncException;
+import org.apache.hudi.sync.common.model.Partition;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.parquet.schema.MessageType;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Base class to sync Hudi tables with Hive based metastores, such as Hive 
server, HMS or managed Hive services.
+ */
+public abstract class AbstractHiveSyncHoodieClient extends 
AbstractSyncHoodieClient {
+
+  protected final HoodieTimeline activeTimeline;
+  protected final HiveSyncConfig syncConfig;
+  protected final Configuration hadoopConf;
+  protected final PartitionValueExtractor partitionValueExtractor;
+
+  public AbstractHiveSyncHoodieClient(HiveSyncConfig syncConfig, Configuration 
hadoopConf, FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, 
syncConfig.useFileListingFromMetadata, syncConfig.withOperationField, fs);
+    this.syncConfig = syncConfig;
+    this.hadoopConf = hadoopConf;
+    this.partitionValueExtractor = 
ReflectionUtils.loadClass(syncConfig.partitionValueExtractorClass);
+    this.activeTimeline = 
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
+  }
+
+  public HoodieTimeline getActiveTimeline() {
+    return activeTimeline;
+  }
+
+  /**
+   * Iterate over the storage partitions and find if there are any new 
partitions that need to be added or updated.
+   * Generate a list of PartitionEvent based on the changes required.
+   */
+  protected List<PartitionEvent> getPartitionEvents(List<Partition> 
tablePartitions, List<String> partitionStoragePartitions, boolean 
isDropPartition) {

Review comment:
       these are copied as is from HoodieHiveClient right? if there are any 
additional changes, let me know. 

##########
File path: 
hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogClient.java
##########
@@ -0,0 +1,474 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.aws.sync;
+
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.CollectionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hive.AbstractHiveSyncHoodieClient;
+import org.apache.hudi.hive.HiveSyncConfig;
+import org.apache.hudi.sync.common.model.Partition;
+
+import com.amazonaws.services.glue.AWSGlue;
+import com.amazonaws.services.glue.AWSGlueClientBuilder;
+import com.amazonaws.services.glue.model.AlreadyExistsException;
+import com.amazonaws.services.glue.model.BatchCreatePartitionRequest;
+import com.amazonaws.services.glue.model.BatchCreatePartitionResult;
+import com.amazonaws.services.glue.model.BatchUpdatePartitionRequest;
+import com.amazonaws.services.glue.model.BatchUpdatePartitionRequestEntry;
+import com.amazonaws.services.glue.model.BatchUpdatePartitionResult;
+import com.amazonaws.services.glue.model.Column;
+import com.amazonaws.services.glue.model.CreateDatabaseRequest;
+import com.amazonaws.services.glue.model.CreateDatabaseResult;
+import com.amazonaws.services.glue.model.CreateTableRequest;
+import com.amazonaws.services.glue.model.CreateTableResult;
+import com.amazonaws.services.glue.model.DatabaseInput;
+import com.amazonaws.services.glue.model.EntityNotFoundException;
+import com.amazonaws.services.glue.model.GetDatabaseRequest;
+import com.amazonaws.services.glue.model.GetPartitionsRequest;
+import com.amazonaws.services.glue.model.GetPartitionsResult;
+import com.amazonaws.services.glue.model.GetTableRequest;
+import com.amazonaws.services.glue.model.PartitionInput;
+import com.amazonaws.services.glue.model.SerDeInfo;
+import com.amazonaws.services.glue.model.StorageDescriptor;
+import com.amazonaws.services.glue.model.Table;
+import com.amazonaws.services.glue.model.TableInput;
+import com.amazonaws.services.glue.model.UpdateTableRequest;
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.parquet.schema.MessageType;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.aws.utils.S3Utils.s3aToS3;
+import static org.apache.hudi.common.util.MapUtils.nonEmpty;
+import static org.apache.hudi.hive.util.HiveSchemaUtil.getPartitionKeyType;
+import static 
org.apache.hudi.hive.util.HiveSchemaUtil.parquetSchemaToMapSchema;
+import static org.apache.hudi.sync.common.util.TableUtils.tableId;
+
+/**
+ * This class implements all the AWS APIs to enable syncing of a Hudi Table 
with the
+ * AWS Glue Data Catalog 
(https://docs.aws.amazon.com/glue/latest/dg/populate-data-catalog.html).
+ */
+public class AWSGlueCatalogClient extends AbstractHiveSyncHoodieClient {
+
+  private static final Logger LOG = 
LogManager.getLogger(AWSGlueCatalogClient.class);
+  private final AWSGlue awsGlue;
+  private final String databaseName;
+
+  public AWSGlueCatalogClient(HiveSyncConfig syncConfig, Configuration 
hadoopConf, FileSystem fs) {
+    super(syncConfig, hadoopConf, fs);
+    this.awsGlue = AWSGlueClientBuilder.standard().build();
+    this.databaseName = syncConfig.databaseName;
+  }
+
+  @Override
+  public List<Partition> getAllPartitions(String tableName) {
+    try {
+      GetPartitionsRequest request = new GetPartitionsRequest();
+      request.withDatabaseName(databaseName).withTableName(tableName);
+      GetPartitionsResult result = awsGlue.getPartitions(request);
+      return result.getPartitions()
+          .stream()
+          .map(p -> new Partition(p.getValues(), 
p.getStorageDescriptor().getLocation()))
+          .collect(Collectors.toList());
+    } catch (Exception e) {
+      throw new HoodieGlueSyncException("Failed to get all partitions for 
table " + tableId(databaseName, tableName), e);
+    }
+  }
+
+  @Override
+  public void addPartitionsToTable(String tableName, List<String> 
partitionsToAdd) {
+    if (partitionsToAdd.isEmpty()) {
+      LOG.info("No partitions to add for " + tableId(databaseName, tableName));
+      return;
+    }
+    LOG.info("Adding " + partitionsToAdd.size() + " partition(s) in table " + 
tableId(databaseName, tableName));
+    try {
+      Table table = getTable(awsGlue, databaseName, tableName);
+      StorageDescriptor sd = table.getStorageDescriptor();
+      List<PartitionInput> partitionInputs = 
partitionsToAdd.stream().map(partition -> {
+        StorageDescriptor partitionSd = sd.clone();
+        String fullPartitionPath = 
FSUtils.getPartitionPath(syncConfig.basePath, partition).toString();
+        List<String> partitionValues = 
partitionValueExtractor.extractPartitionValuesInPath(partition);
+        partitionSd.setLocation(fullPartitionPath);
+        return new 
PartitionInput().withValues(partitionValues).withStorageDescriptor(partitionSd);
+      }).collect(Collectors.toList());
+
+      BatchCreatePartitionRequest request = new BatchCreatePartitionRequest();
+      
request.withDatabaseName(databaseName).withTableName(tableName).withPartitionInputList(partitionInputs);
+
+      BatchCreatePartitionResult result = 
awsGlue.batchCreatePartition(request);
+      if (CollectionUtils.nonEmpty(result.getErrors())) {
+        throw new HoodieGlueSyncException("Fail to add partitions to " + 
tableId(databaseName, tableName)
+            + " with error(s): " + result.getErrors());
+      }
+    } catch (Exception e) {
+      throw new HoodieGlueSyncException("Fail to add partitions to " + 
tableId(databaseName, tableName), e);
+    }
+  }
+
+  @Override
+  public void updatePartitionsToTable(String tableName, List<String> 
changedPartitions) {
+    if (changedPartitions.isEmpty()) {
+      LOG.info("No partitions to change for " + tableName);
+      return;
+    }
+    LOG.info("Updating " + changedPartitions.size() + "partition(s) in table " 
+ tableId(databaseName, tableName));
+    try {
+      Table table = getTable(awsGlue, databaseName, tableName);
+      StorageDescriptor sd = table.getStorageDescriptor();
+      List<BatchUpdatePartitionRequestEntry> updatePartitionEntries = 
changedPartitions.stream().map(partition -> {
+        StorageDescriptor partitionSd = sd.clone();
+        String fullPartitionPath = 
FSUtils.getPartitionPath(syncConfig.basePath, partition).toString();
+        List<String> partitionValues = 
partitionValueExtractor.extractPartitionValuesInPath(partition);
+        sd.setLocation(fullPartitionPath);
+        PartitionInput partitionInput = new 
PartitionInput().withValues(partitionValues).withStorageDescriptor(partitionSd);
+        return new 
BatchUpdatePartitionRequestEntry().withPartitionInput(partitionInput).withPartitionValueList(partitionValues);
+      }).collect(Collectors.toList());
+
+      BatchUpdatePartitionRequest request = new BatchUpdatePartitionRequest();
+      
request.withDatabaseName(databaseName).withTableName(tableName).withEntries(updatePartitionEntries);
+
+      BatchUpdatePartitionResult result = 
awsGlue.batchUpdatePartition(request);
+      if (CollectionUtils.nonEmpty(result.getErrors())) {
+        throw new HoodieGlueSyncException("Fail to update partitions to " + 
tableId(databaseName, tableName)
+            + " with error(s): " + result.getErrors());
+      }
+    } catch (Exception e) {
+      throw new HoodieGlueSyncException("Fail to update partitions to " + 
tableId(databaseName, tableName), e);
+    }
+  }
+
+  @Override
+  public void dropPartitionsToTable(String tableName, List<String> 
partitionsToDrop) {
+    throw new UnsupportedOperationException("Not support dropPartitionsToTable 
yet.");
+  }
+
+  /**
+   * Update the table properties to the table.
+   */
+  @Override
+  public void updateTableProperties(String tableName, Map<String, String> 
tableProperties) {
+    if (nonEmpty(tableProperties)) {
+      return;
+    }
+    try {
+      updateTableParameters(awsGlue, databaseName, tableName, tableProperties, 
true);
+    } catch (Exception e) {
+      throw new HoodieGlueSyncException("Fail to update properties for table " 
+ tableId(databaseName, tableName), e);
+    }
+  }
+
+  @Override
+  public void updateTableDefinition(String tableName, MessageType newSchema) {
+    // ToDo Cascade is set in Hive meta sync, but need to investigate how to 
configure it for Glue meta
+    boolean cascade = syncConfig.partitionFields.size() > 0;
+    try {
+      Table table = getTable(awsGlue, databaseName, tableName);
+      Map<String, String> newSchemaMap = parquetSchemaToMapSchema(newSchema, 
syncConfig.supportTimestamp, false);
+      List<Column> newColumns = newSchemaMap.keySet().stream().map(key -> {
+        String keyType = getPartitionKeyType(newSchemaMap, key);
+        return new 
Column().withName(key).withType(keyType.toLowerCase()).withComment("");
+      }).collect(Collectors.toList());
+      StorageDescriptor sd = table.getStorageDescriptor();
+      sd.setColumns(newColumns);
+
+      final Date now = new Date();
+      TableInput updatedTableInput = new TableInput()

Review comment:
       I see we instantiate this table in similar fashion in few methods in 
this class. Do you think we can make a private method. 
   not too strong on the suggestion. I will let you take a call. 
   




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to