You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/05/02 18:27:25 UTC

[GitHub] [spark] sunchao commented on a diff in pull request #36427: [SPARK-39086][SQL] Support UDT in Spark Parquet vectorized reader

sunchao commented on code in PR #36427:
URL: https://github.com/apache/spark/pull/36427#discussion_r863072081


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala:
##########
@@ -208,6 +208,9 @@ object ParquetUtils {
     case st: StructType =>
       sqlConf.parquetVectorizedReaderNestedColumnEnabled &&
         st.fields.forall(f => isBatchReadSupported(sqlConf, f.dataType))
+    case udt: UserDefinedType[_] =>
+      sqlConf.parquetVectorizedReaderNestedColumnEnabled &&

Review Comment:
   why do we need to check `sqlConf.parquetVectorizedReaderNestedColumnEnabled`? if `udt.sqlType` is `AtomicType` then this should return true.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala:
##########
@@ -174,9 +174,15 @@ class ParquetToSparkSchemaConverter(
    */
   def convertField(
       field: ColumnIO,
-      sparkReadType: Option[DataType] = None): ParquetColumn = field match {
-    case primitiveColumn: PrimitiveColumnIO => convertPrimitiveField(primitiveColumn, sparkReadType)
-    case groupColumn: GroupColumnIO => convertGroupField(groupColumn, sparkReadType)
+      sparkReadType: Option[DataType] = None): ParquetColumn = {
+    val targetType = sparkReadType.map {

Review Comment:
   I think there is another edge case in `convertInternal` when `fieldReadType` is `ArrayType` too - maybe we also need to check if the field type is UDT whose `sqlType` is `ArrayType`.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org