You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/06/21 20:20:39 UTC

[GitHub] [hudi] satishkotha commented on a change in pull request #2879: [HUDI-1848] Adding support for HMS for running DDL queries in hive-sy…

satishkotha commented on a change in pull request #2879:
URL: https://github.com/apache/hudi/pull/2879#discussion_r655675336



##########
File path: hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.ddl;
+
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.fs.StorageSchemes;
+import org.apache.hudi.hive.HiveSyncConfig;
+import org.apache.hudi.hive.HoodieHiveSyncException;
+import org.apache.hudi.hive.PartitionValueExtractor;
+import org.apache.hudi.hive.util.HiveSchemaUtil;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.parquet.schema.MessageType;
+import org.apache.thrift.TException;
+
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class HMSDDLExecutor implements DDLExecutor {
+  private static final Logger LOG = LogManager.getLogger(HMSDDLExecutor.class);
+  private final HiveSyncConfig syncConfig;
+  private final PartitionValueExtractor partitionValueExtractor;
+  private final FileSystem fs;
+  private IMetaStoreClient client;
+
+  public HMSDDLExecutor(HiveConf conf, HiveSyncConfig syncConfig, FileSystem fs) throws HiveException, MetaException {
+    this.client = Hive.get(conf).getMSC();
+    this.syncConfig = syncConfig;
+    this.fs = fs;
+    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 void createDatabase(String databaseName) {
+    try {
+      Database database = new Database(databaseName, "automatically created by hoodie", null, null);
+      client.createDatabase(database);
+    } catch (Exception e) {
+      LOG.error("Failed to create database " + databaseName, e);
+      throw new HoodieHiveSyncException("Failed to create database " + databaseName, e);
+    }
+  }
+
+  @Override
+  public void createTable(String tableName, MessageType storageSchema, String inputFormatClass, String outputFormatClass, String serdeClass, Map<String, String> serdeProperties,
+                          Map<String, String> tableProperties) {
+    try {
+      LinkedHashMap<String, String> mapSchema = HiveSchemaUtil.parquetSchemaToMapSchema(storageSchema, syncConfig.supportTimestamp);
+
+      List<FieldSchema> fieldSchema = HiveSchemaUtil.convertMapSchemaToHiveFieldSchema(mapSchema, syncConfig);
+      Map<String, String> hiveSchema = HiveSchemaUtil.convertMapSchemaToHiveSchema(mapSchema);
+
+      List<FieldSchema> partitionSchema = syncConfig.partitionFields.stream().map(partitionKey -> {
+        String partitionKeyType = HiveSchemaUtil.getPartitionKeyType(hiveSchema, partitionKey);

Review comment:
       Can we change getPartitionKeyType to work with fieldSchema/mapSchema? can we remove 'hiveSchema' variable with that?

##########
File path: hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java
##########
@@ -138,8 +140,28 @@ private static boolean isFieldExistsInSchema(Map<String, String> newTableSchema,
    * @param messageType : Parquet Schema
    * @return : Hive Table schema read from parquet file MAP[String,String]
    */
-  private static Map<String, String> convertParquetSchemaToHiveSchema(MessageType messageType, boolean supportTimestamp) throws IOException {
-    Map<String, String> schema = new LinkedHashMap<>();
+  public static Map<String, String> convertParquetSchemaToHiveSchema(MessageType messageType, boolean supportTimestamp) throws IOException {
+    return convertMapSchemaToHiveSchema(parquetSchemaToMapSchema(messageType, supportTimestamp));

Review comment:
       It seems like we are always doing two translations
   parquet -> map and map -> hive schema? Is this extra step needed for all executors?

##########
File path: hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java
##########
@@ -150,11 +172,26 @@ private static boolean isFieldExistsInSchema(Map<String, String> newTableSchema,
         result.append(convertField(parquetType, supportTimestamp));
       }
 
-      schema.put(hiveCompatibleFieldName(key, false), result.toString());
+      schema.put(key, result.toString());
     }
     return schema;
   }
 
+  public static Map<String, String> convertMapSchemaToHiveSchema(LinkedHashMap<String, String> schema) throws IOException {
+    Map<String, String> hiveSchema = new LinkedHashMap<>();
+    for (Map.Entry<String,String> entry: schema.entrySet()) {
+      hiveSchema.put(hiveCompatibleFieldName(entry.getKey(), false), entry.getValue());
+    }
+    return hiveSchema;
+  }
+
+  public static List<FieldSchema> convertMapSchemaToHiveFieldSchema(LinkedHashMap<String, String> schema, HiveSyncConfig syncConfig) throws IOException {
+    return schema.keySet().stream()
+        .map(key -> new FieldSchema(key, schema.get(key), ""))
+        .filter(field -> !syncConfig.partitionFields.contains(field.getName()))

Review comment:
       why is this extra filter required? I cant seem to find this in previous version of convertParquetSchemaToHiveSchema method.

##########
File path: hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.ddl;
+
+import org.apache.parquet.schema.MessageType;
+
+import java.util.List;
+import java.util.Map;
+
+public interface DDLExecutor {

Review comment:
       @jsbali you may have missed this




-- 
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:
users@infra.apache.org