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 2020/06/09 16:12:52 UTC

[GitHub] [spark] Ngone51 commented on a change in pull request #28645: [SPARK-31826][SQL] Support composed type of case class for typed Scala UDF

Ngone51 commented on a change in pull request #28645:
URL: https://github.com/apache/spark/pull/28645#discussion_r437108708



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2847,6 +2848,36 @@ class Analyzer(
     }
   }
 
+  object ResolveEncodersInUDF extends Rule[LogicalPlan] {
+    override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+      case p if !p.resolved => p // Skip unresolved nodes.
+
+      case p => p transformExpressionsUp {
+
+        case udf @ ScalaUDF(_, _, inputs, encoders, _, _, _) if encoders.nonEmpty =>
+          val resolvedEncoders = encoders.zipWithIndex.map { case (encOpt, i) =>
+            val dataType = inputs(i).dataType
+            if (dataType.isInstanceOf[UserDefinedType[_]]) {
+              // for UDT, we use `CatalystTypeConverters`
+              None
+            } else {
+              encOpt.map { enc =>
+                val attrs = if (enc.isSerializedAsStructForTopLevel) {
+                  dataType.asInstanceOf[StructType].toAttributes
+                } else {
+                  // the field name doesn't matter here, so we use
+                  // a simple literal to avoid any overhead
+                  new StructType().add("input", dataType).toAttributes
+                }

Review comment:
       By using internal `schema`, `ExpressionEncoder` would always use `Decimal(38, 18)` as the data type for the input decimal value when deserializing an internal row to external value. However, it could cause data loss when the data type of the input decimal value is not compatible with `Decimal(38, 18)`. For example, we should not cast the input value to `Deciaml(38, 18)` if the actual data type is `Decimal(30, 0)`. Therefore, we give the attrs explicitly here in order to let the encoder uses the actual data type from the input value when doing deserialzation.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2847,6 +2848,36 @@ class Analyzer(
     }
   }
 
+  object ResolveEncodersInUDF extends Rule[LogicalPlan] {

Review comment:
       sure




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



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