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


##########
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:
   I guess without stats and constraints (when you cannot build the valid map), 
the query should still be executed to the same result as they are only for 
optimizing query. So I think even for the case, we still can run the query 
instead of failing it.
   
   And, even you fail the query, I'm not sure what users can do to make valid 
stats and constraints avaiable?



##########
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:
   I guess without stats and constraints (when you cannot build the valid map), 
the query should still be executed to the same result as they are only for 
optimizing query. So I think even for the case, we still can run the query 
instead of failing it.
   
   And, even you fail the query, I'm not sure what users can do to make valid 
stats and constraints available?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to