xushiyan commented on a change in pull request #4080:
URL: https://github.com/apache/hudi/pull/4080#discussion_r756467883



##########
File path: 
hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HoodieGlueClient.java
##########
@@ -0,0 +1,487 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.util;
+
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hive.AbstractHiveSyncHoodieClient;
+import org.apache.hudi.hive.HiveSyncConfig;
+import org.apache.hudi.hive.HoodieHiveSyncException;
+import org.apache.hudi.hive.PartitionValueExtractor;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+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.Partition;
+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 com.esotericsoftware.minlog.Log;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+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.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Currently Experimental. This class implements the logic to sync meta with 
the AWS Glue (managed Hive) server
+ * to enable querying via Glue ETLs, Athena etc.
+ */
+public class HoodieGlueClient extends AbstractHiveSyncHoodieClient {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieGlueClient.class);
+  private final HoodieTimeline activeTimeline;
+  private final AWSGlue awsGlueClient;
+  private final HiveSyncConfig syncConfig;
+  private final PartitionValueExtractor partitionValueExtractor;
+
+  public HoodieGlueClient(HiveSyncConfig cfg, FileSystem fs) {
+    super(cfg.basePath, cfg.assumeDatePartitioning, 
cfg.useFileListingFromMetadata, cfg.withOperationField, fs);
+    this.awsGlueClient = getGlueClient();
+    this.syncConfig = cfg;
+    activeTimeline = 
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
+    try {
+      this.partitionValueExtractor =
+          (PartitionValueExtractor) 
Class.forName(syncConfig.partitionValueExtractorClass).newInstance();
+    } catch (Exception e) {
+      throw new HoodieHiveSyncException(
+          "Failed to initialize PartitionValueExtractor class " + 
syncConfig.partitionValueExtractorClass, e);
+    }
+  }
+
+  @Override
+  public boolean doesDataBaseExist(String databaseName) {
+    GetDatabaseRequest request = new GetDatabaseRequest();
+    request.setName(databaseName);
+    try {
+      return (awsGlueClient.getDatabase(request).getDatabase() != null);
+    } catch (EntityNotFoundException exception) {
+      LOG.error("Database " + databaseName, exception);
+    } catch (Exception exception) {
+      LOG.error("Failed to check if database exists " + databaseName, 
exception);
+      throw new HoodieHiveSyncException("Failed to check if database exists " 
+ databaseName
+          + " in region ", exception);
+    }
+    return false;
+  }
+
+  @Override
+  public void createDatabase(String databaseName) {
+    if (!doesDataBaseExist(databaseName)) {
+      CreateDatabaseRequest request = new CreateDatabaseRequest();
+      request.setDatabaseInput(new 
DatabaseInput().withName(databaseName).withDescription("automatically created 
by hudi").withParameters(null).withLocationUri(null));
+      try {
+        CreateDatabaseResult result = awsGlueClient.createDatabase(request);
+        LOG.info("Successfully created database in Glue: " + 
result.toString());
+      } catch (AlreadyExistsException exception) {
+        LOG.warn("Database " + databaseName + " already exists", exception);
+      } catch (Exception exception) {
+        LOG.error("Failed to create database " + databaseName, exception);
+        throw new HoodieHiveSyncException("Failed to create database " + 
databaseName, exception);
+      }
+    }
+  }
+
+  @Override
+  public boolean doesTableExist(String tableName) {
+    GetTableRequest request = new GetTableRequest()
+        .withDatabaseName(syncConfig.databaseName)
+        .withName(tableName);
+    try {
+      awsGlueClient.getTable(request);
+      return true;
+    } catch (EntityNotFoundException exception) {
+      LOG.error("Accessing non-existent Glue Table " + tableName + " in 
database " + syncConfig.databaseName, exception);
+    } catch (Exception exception) {
+      String errorMsg = "Fatal error while fetching Glue Table " + tableName + 
" in database " + syncConfig.databaseName;
+      LOG.error(errorMsg, exception);
+      throw new HoodieHiveSyncException(errorMsg, exception);
+    }
+    return false;
+  }
+
+  @Override
+  public void createTable(String tableName, MessageType storageSchema,
+                          String inputFormatClass, String outputFormatClass,
+                          String serdeClass, Map<String, String> 
serdeProperties,
+                          Map<String, String> tableProperties) {
+    if (!doesTableExist(tableName)) {
+      CreateTableRequest request = new CreateTableRequest();
+      Map<String, String> params = new HashMap<>();
+      if (!syncConfig.createManagedTable) {
+        params.put("EXTERNAL", "TRUE");
+      }
+      for (Map.Entry<String, String> entry : tableProperties.entrySet()) {
+        params.put(entry.getKey(), entry.getValue());
+      }
+
+      try {
+        LinkedHashMap<String, String> mapSchema = 
HiveSchemaUtil.parquetSchemaToMapSchema(storageSchema, 
syncConfig.supportTimestamp, false);
+
+        List<Column> schemaPartitionKeys = new ArrayList<>();
+        List<Column> schemaWithoutPartitionKeys = new ArrayList<>();
+        for (String key : mapSchema.keySet()) {
+          String keyType = HiveSchemaUtil.getPartitionKeyType(mapSchema, key);
+          Column column = new 
Column().withName(key).withType(keyType.toLowerCase()).withComment("");
+          // In Glue, the full schema should exclude the partition keys
+          if (syncConfig.partitionFields.contains(key)) {
+            schemaPartitionKeys.add(column);
+          } else {
+            schemaWithoutPartitionKeys.add(column);
+          }
+        }
+
+        StorageDescriptor storageDescriptor = new StorageDescriptor();
+        serdeProperties.put("serialization.format", "1");
+        storageDescriptor
+            .withSerdeInfo(new SerDeInfo().withSerializationLibrary(serdeClass)
+                .withParameters(serdeProperties))
+            .withLocation(syncConfig.basePath.replaceFirst("s3a", "s3"))
+            .withInputFormat(inputFormatClass)
+            .withOutputFormat(outputFormatClass)
+            .withColumns(schemaWithoutPartitionKeys);
+
+        TableInput tableInput = new TableInput();
+        tableInput.withName(tableName)
+            .withTableType(TableType.EXTERNAL_TABLE.toString())
+            .withParameters(params)
+            .withPartitionKeys(schemaPartitionKeys)
+            .withStorageDescriptor(storageDescriptor)
+            .withLastAccessTime(new Date(System.currentTimeMillis()))
+            .withLastAnalyzedTime(new Date(System.currentTimeMillis()));
+        request.withDatabaseName(syncConfig.databaseName)
+            .withTableInput(tableInput);
+
+        CreateTableResult result = awsGlueClient.createTable(request);
+        LOG.info("Successfully created table in Glue: " + result.toString());
+      } catch (AlreadyExistsException exception) {
+        LOG.warn("Table " + tableName + " already exists in database " + 
syncConfig.databaseName, exception);
+      } catch (Exception exception) {
+        LOG.error("Failed to create table " + tableName + " in database " + 
syncConfig.databaseName, exception);
+        throw new HoodieHiveSyncException("Failed to create table " + 
tableName + " in database " + syncConfig.databaseName, exception);
+      }
+    }
+  }
+
+  @Override
+  public Option<String> getLastCommitTimeSynced(String tableName) {
+    try {
+      Table table = getTable(syncConfig.databaseName, tableName);
+      return 
Option.of(table.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, 
null));

Review comment:
       ```suggestion
         return 
Option.ofNullable(table.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC,
 null));
   ```

##########
File path: 
hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HoodieGlueClient.java
##########
@@ -0,0 +1,487 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.util;
+
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hive.AbstractHiveSyncHoodieClient;
+import org.apache.hudi.hive.HiveSyncConfig;
+import org.apache.hudi.hive.HoodieHiveSyncException;
+import org.apache.hudi.hive.PartitionValueExtractor;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+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.Partition;
+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 com.esotericsoftware.minlog.Log;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+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.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Currently Experimental. This class implements the logic to sync meta with 
the AWS Glue (managed Hive) server
+ * to enable querying via Glue ETLs, Athena etc.
+ */
+public class HoodieGlueClient extends AbstractHiveSyncHoodieClient {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieGlueClient.class);
+  private final HoodieTimeline activeTimeline;
+  private final AWSGlue awsGlueClient;
+  private final HiveSyncConfig syncConfig;
+  private final PartitionValueExtractor partitionValueExtractor;
+
+  public HoodieGlueClient(HiveSyncConfig cfg, FileSystem fs) {
+    super(cfg.basePath, cfg.assumeDatePartitioning, 
cfg.useFileListingFromMetadata, cfg.withOperationField, fs);
+    this.awsGlueClient = getGlueClient();
+    this.syncConfig = cfg;
+    activeTimeline = 
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
+    try {
+      this.partitionValueExtractor =
+          (PartitionValueExtractor) 
Class.forName(syncConfig.partitionValueExtractorClass).newInstance();
+    } catch (Exception e) {
+      throw new HoodieHiveSyncException(
+          "Failed to initialize PartitionValueExtractor class " + 
syncConfig.partitionValueExtractorClass, e);
+    }
+  }
+
+  @Override
+  public boolean doesDataBaseExist(String databaseName) {
+    GetDatabaseRequest request = new GetDatabaseRequest();
+    request.setName(databaseName);
+    try {
+      return (awsGlueClient.getDatabase(request).getDatabase() != null);
+    } catch (EntityNotFoundException exception) {
+      LOG.error("Database " + databaseName, exception);
+    } catch (Exception exception) {
+      LOG.error("Failed to check if database exists " + databaseName, 
exception);
+      throw new HoodieHiveSyncException("Failed to check if database exists " 
+ databaseName
+          + " in region ", exception);
+    }
+    return false;
+  }
+
+  @Override
+  public void createDatabase(String databaseName) {
+    if (!doesDataBaseExist(databaseName)) {
+      CreateDatabaseRequest request = new CreateDatabaseRequest();
+      request.setDatabaseInput(new 
DatabaseInput().withName(databaseName).withDescription("automatically created 
by hudi").withParameters(null).withLocationUri(null));
+      try {
+        CreateDatabaseResult result = awsGlueClient.createDatabase(request);
+        LOG.info("Successfully created database in Glue: " + 
result.toString());
+      } catch (AlreadyExistsException exception) {
+        LOG.warn("Database " + databaseName + " already exists", exception);
+      } catch (Exception exception) {
+        LOG.error("Failed to create database " + databaseName, exception);
+        throw new HoodieHiveSyncException("Failed to create database " + 
databaseName, exception);
+      }
+    }
+  }
+
+  @Override
+  public boolean doesTableExist(String tableName) {
+    GetTableRequest request = new GetTableRequest()
+        .withDatabaseName(syncConfig.databaseName)
+        .withName(tableName);
+    try {
+      awsGlueClient.getTable(request);
+      return true;
+    } catch (EntityNotFoundException exception) {
+      LOG.error("Accessing non-existent Glue Table " + tableName + " in 
database " + syncConfig.databaseName, exception);
+    } catch (Exception exception) {
+      String errorMsg = "Fatal error while fetching Glue Table " + tableName + 
" in database " + syncConfig.databaseName;
+      LOG.error(errorMsg, exception);
+      throw new HoodieHiveSyncException(errorMsg, exception);
+    }
+    return false;
+  }
+
+  @Override
+  public void createTable(String tableName, MessageType storageSchema,
+                          String inputFormatClass, String outputFormatClass,
+                          String serdeClass, Map<String, String> 
serdeProperties,
+                          Map<String, String> tableProperties) {
+    if (!doesTableExist(tableName)) {
+      CreateTableRequest request = new CreateTableRequest();
+      Map<String, String> params = new HashMap<>();
+      if (!syncConfig.createManagedTable) {
+        params.put("EXTERNAL", "TRUE");
+      }
+      for (Map.Entry<String, String> entry : tableProperties.entrySet()) {
+        params.put(entry.getKey(), entry.getValue());
+      }
+
+      try {
+        LinkedHashMap<String, String> mapSchema = 
HiveSchemaUtil.parquetSchemaToMapSchema(storageSchema, 
syncConfig.supportTimestamp, false);
+
+        List<Column> schemaPartitionKeys = new ArrayList<>();
+        List<Column> schemaWithoutPartitionKeys = new ArrayList<>();
+        for (String key : mapSchema.keySet()) {
+          String keyType = HiveSchemaUtil.getPartitionKeyType(mapSchema, key);
+          Column column = new 
Column().withName(key).withType(keyType.toLowerCase()).withComment("");
+          // In Glue, the full schema should exclude the partition keys
+          if (syncConfig.partitionFields.contains(key)) {
+            schemaPartitionKeys.add(column);
+          } else {
+            schemaWithoutPartitionKeys.add(column);
+          }
+        }
+
+        StorageDescriptor storageDescriptor = new StorageDescriptor();
+        serdeProperties.put("serialization.format", "1");
+        storageDescriptor
+            .withSerdeInfo(new SerDeInfo().withSerializationLibrary(serdeClass)
+                .withParameters(serdeProperties))
+            .withLocation(syncConfig.basePath.replaceFirst("s3a", "s3"))
+            .withInputFormat(inputFormatClass)
+            .withOutputFormat(outputFormatClass)
+            .withColumns(schemaWithoutPartitionKeys);
+
+        TableInput tableInput = new TableInput();
+        tableInput.withName(tableName)
+            .withTableType(TableType.EXTERNAL_TABLE.toString())
+            .withParameters(params)
+            .withPartitionKeys(schemaPartitionKeys)
+            .withStorageDescriptor(storageDescriptor)
+            .withLastAccessTime(new Date(System.currentTimeMillis()))
+            .withLastAnalyzedTime(new Date(System.currentTimeMillis()));
+        request.withDatabaseName(syncConfig.databaseName)
+            .withTableInput(tableInput);
+
+        CreateTableResult result = awsGlueClient.createTable(request);
+        LOG.info("Successfully created table in Glue: " + result.toString());
+      } catch (AlreadyExistsException exception) {
+        LOG.warn("Table " + tableName + " already exists in database " + 
syncConfig.databaseName, exception);
+      } catch (Exception exception) {
+        LOG.error("Failed to create table " + tableName + " in database " + 
syncConfig.databaseName, exception);
+        throw new HoodieHiveSyncException("Failed to create table " + 
tableName + " in database " + syncConfig.databaseName, exception);
+      }
+    }
+  }
+
+  @Override
+  public Option<String> getLastCommitTimeSynced(String tableName) {
+    try {
+      Table table = getTable(syncConfig.databaseName, tableName);
+      return 
Option.of(table.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, 
null));
+    } catch (Exception e) {
+      throw new HoodieHiveSyncException("Failed to get the last commit time 
synced from the database", e);
+    }
+  }
+
+  @Override
+  public void updateLastCommitTimeSynced(String tableName) {

Review comment:
       can you double check and ensure caller to this method respects 
`--conditional-sync` flag in HiveSyncConfig? 




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