[ 
https://issues.apache.org/jira/browse/HUDI-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17378914#comment-17378914
 ] 

ASF GitHub Bot commented on HUDI-2045:
--------------------------------------

danny0405 commented on a change in pull request #3120:
URL: https://github.com/apache/hudi/pull/3120#discussion_r667613444



##########
File path: 
hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/Parquet2SparkSchemaUtils.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.util.ValidationUtils;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+import static org.apache.parquet.schema.Type.Repetition.OPTIONAL;
+
+/**
+ * Convert the parquet schema to spark schema' json string.
+ * This code is refer to 
org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter
+ * in spark project.
+ */

Review comment:
       Why not just use `ParquetToSparkSchemaConverter` directly.

##########
File path: 
hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java
##########
@@ -70,6 +69,10 @@
     return Arrays.asList(new Object[][] {{true, true, true}, {true, false, 
false}, {false, true, true}, {false, false, false}});
   }
 
+  private static Iterable<Object[]> syncDataSourceTableParams() {
+    return Arrays.asList(new Object[][] {{true, true, true}, {true, false, 
false}, {false, true, true}, {false, false, false}});

Review comment:
       Can we give some comments what each flag means ?

##########
File path: 
hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/ConfigUtils.java
##########
@@ -23,12 +23,11 @@
 import org.apache.hudi.common.util.StringUtils;
 
 public class ConfigUtils {
-
-  public static final String SPARK_QUERY_TYPE_KEY = "spark.query.type.key";
-
-  public static final String SPARK_QUERY_AS_RO_KEY = "spark.query.as.ro.key";
-
-  public static final String SPARK_QUERY_AS_RT_KEY = "spark.query.as.rt.key";
+  /**
+   * Config stored in hive serde properties to tell query engine (spark/flink) 
to
+   * read the table as a read-optimized table when this config is true.
+   */
+  public static final String IS_QUERY_AS_RO_TABLE = "hoodie.query.as.ro.table";
 

Review comment:
       What's the relationship between this key `IS_QUERY_AS_RO_TABLE` and 
`SPARK_QUERY_AS_RO_KEY ` and `SPARK_QUERY_AS_RT_KEY `.




-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Support Read Hoodie As DataSource Table For Flink And DeltaStreamer
> -------------------------------------------------------------------
>
>                 Key: HUDI-2045
>                 URL: https://issues.apache.org/jira/browse/HUDI-2045
>             Project: Apache Hudi
>          Issue Type: Improvement
>          Components: Hive Integration
>            Reporter: pengzhiwei
>            Assignee: pengzhiwei
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 0.9.0
>
>
> Currently we only support reading hoodie table as datasource table for spark 
> since [https://github.com/apache/hudi/pull/2283]
> In order to support this feature for flink and DeltaStreamer, we need to sync 
> the spark table properties needed by datasource table to the meta store in 
> HiveSyncTool.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to