You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "YannisSismanis (via GitHub)" <gi...@apache.org> on 2023/10/03 18:59:15 UTC

Re: [PR] [SPARK-44219][SQL] Adds extra per-rule validations for optimization rewrites. [spark]

YannisSismanis commented on code in PR #41763:
URL: https://github.com/apache/spark/pull/41763#discussion_r1344584317


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala:
##########
@@ -324,6 +325,124 @@ object LogicalPlanIntegrity {
       LogicalPlanIntegrity.hasUniqueExprIdsForOutput(plan))
   }
 
+  /**
+   * This method validates there are no dangling attribute references.
+   * Returns an error message if the check does not pass, or None if it does pass.
+   */
+  def validateNoDanglingReferences(plan: LogicalPlan): Option[String] = {
+    plan.collectFirst {
+      // DML commands and multi instance relations (like InMemoryRelation caches)
+      // have different output semantics than typical queries.
+      case _: Command => None
+      case _: MultiInstanceRelation => None
+      case n if canGetOutputAttrs(n) =>
+        val inputExprIds = (n.children.flatMap(c => c.output ++ c.metadataOutput) ++ n.output)
+          .map(_.exprId).toSet
+        val danglingReferences = n.references.filter {
+          a => a.resolved && !inputExprIds.contains(a.exprId)
+        }.map(_.qualifiedName)
+        if (danglingReferences.nonEmpty) {
+          Some(s"Aliases ${danglingReferences.mkString(", ")} are dangling " +
+            s"in the references for plan:\n ${n.treeString}")
+        } else {
+          None
+        }
+    }.flatten
+  }
+
+  /**
+   * Validate that the grouping key types in Aggregate plans are valid.
+   * Returns an error message if the check fails, or None if it succeeds.
+   */
+  def validateGroupByTypes(plan: LogicalPlan): Option[String] = {
+    plan.collectFirst {
+      case a @ Aggregate(groupingExprs, _, _) =>
+        val badExprs = groupingExprs.filter(_.dataType.isInstanceOf[MapType]).map(_.toString)
+        if (badExprs.nonEmpty) {
+          Some(s"Grouping expressions ${badExprs.mkString(", ")} cannot be of type Map " +
+            s"for plan:\n ${a.treeString}")
+        } else {
+          None
+        }
+    }.flatten
+  }
+
+  /**
+   * Validate that the aggregation expressions in Aggregate plans are valid.
+   * Returns an error message if the check fails, or None if it succeeds.
+   */
+  def validateAggregateExpressions(plan: LogicalPlan): Option[String] = {
+    /**
+    * Returns true if all the exprIds, referenced in an Expression <subExpr>
+    * (used in the aggregate expressions of an Aggregate),
+    * are either used as measures (under an AggregateFunction) or are explicitly grouped,
+    * given the <groupingExprIds> of all grouping expressions
+    * (used in the grouping expressions of the same Aggregate)
+
+    * I.e:
+    * <groupingExprIds> can be used "freely" in an AggregateExpressions, all other exprIds can only
+    * be used under an AggregateFunction (i.e. aggregated).
+     */
+    def isAggregateSubExpressionValid(
+          groupingExpressions: Seq[Expression],
+          groupingExprIds: Set[ExprId],
+          subExpr: Expression): Boolean = {
+      // First collect all non-grouping ExprIds in <subExpr>.
+      val restrictedExprIds = subExpr.collect {
+        case g: Attribute if !groupingExprIds.contains(g.exprId) => g.exprId
+      }.groupBy(identity).mapValues(_.size)
+
+      // Second collect all non-grouping ExprIds under an AggregateFunction in <subExpr>.
+      val restrictedExprIdsUnderAggregateFunction = subExpr.flatMap {
+        case a: AggregateExpression => a.collect {
+          case g: Attribute if !groupingExprIds.contains(g.exprId) => g.exprId
+        }
+        case _ => Seq.empty
+      }.groupBy(identity).mapValues(_.size)
+
+      // Validate a) that all non-grouping exprIds are used under some aggregate function,
+      // i.e. all such exprIds are aggregated (measures) OR
+      // b) a grouping expression can be used in sub-expressions
+      // for aggregate expressions of an Aggregate.
+      restrictedExprIds == restrictedExprIdsUnderAggregateFunction ||

Review Comment:
   synced offline



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