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/12/16 03:00:05 UTC

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #39082: [SPARK-41539][SQL] Remap stats and constraints against output in logical plan for LogicalRDD

HeartSaVioR commented on code in PR #39082:
URL: https://github.com/apache/spark/pull/39082#discussion_r1050326351


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala:
##########
@@ -183,16 +172,80 @@ object LogicalRDD {
       }
     }
 
+    val logicalPlan = originDataset.logicalPlan
     val optimizedPlan = originDataset.queryExecution.optimizedPlan
     val executedPlan = originDataset.queryExecution.executedPlan
 
+    val (stats, constraints) = rewriteStatsAndConstraints(logicalPlan, optimizedPlan)
+
     LogicalRDD(
       originDataset.logicalPlan.output,
       rdd,
       firstLeafPartitioning(executedPlan.outputPartitioning),
       executedPlan.outputOrdering,
       isStreaming
-    )(originDataset.sparkSession, Some(optimizedPlan.stats), Some(optimizedPlan.constraints))
+    )(originDataset.sparkSession, stats, constraints)
+  }
+
+  private[sql] def buildOutputAssocForRewrite(
+      source: Seq[Attribute],
+      destination: Seq[Attribute]): Option[Map[Attribute, Attribute]] = {
+    // We check the name and type, allowing nullability, exprId, metadata, qualifier be different
+    // E.g. This could happen during optimization phase.
+    val rewrite = source.zip(destination).flatMap { case (attr1, attr2) =>
+      if (attr1.name == attr2.name && attr1.dataType == attr2.dataType) {
+        Some(attr1 -> attr2)
+      } else {
+        None
+      }
+    }.toMap
+
+    if (rewrite.size == source.size) {
+      Some(rewrite)
+    } else {
+      None
+    }
+  }
+
+  private[sql] def rewriteStatsAndConstraints(
+      logicalPlan: LogicalPlan,
+      optimizedPlan: LogicalPlan): (Option[Statistics], Option[ExpressionSet]) = {
+    val rewrite = buildOutputAssocForRewrite(optimizedPlan.output, logicalPlan.output)
+
+    rewrite.map { rw =>
+      val rewrittenStatistics = rewriteStatistics(optimizedPlan.stats, rw)
+      val rewrittenConstraints = rewriteConstraints(optimizedPlan.constraints, rw)
+
+      (Some(rewrittenStatistics), Some(rewrittenConstraints))
+    }.getOrElse {
+      // can't rewrite stats and constraints, give up
+      logWarning("The output columns are expected to the same (for name and type) for output " +

Review Comment:
   This is something I want to discuss. I see the difference of exprId which makes sense, but is it possible to see other difference as well? If it's possible, is LogicalRDD instance even safe?
   
   Suppose we are here, should we raise an internal error instead, or it is still a valid case and we log and give up?



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