You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by frreiss <gi...@git.apache.org> on 2016/05/17 21:11:25 UTC

[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

GitHub user frreiss opened a pull request:

    https://github.com/apache/spark/pull/13155

    [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubquery rule to fix COUNT bug

    ## What changes were proposed in this pull request?
    This pull request fixes the COUNT bug in the `RewriteCorrelatedScalarSubquery` rule.
    
    After this change, the rule tests the expression at the root of the correlated subquery to determine whether the expression returns NULL on empty input. If the expression does not return NULL, the rule generates additional logic in the Project operator above the rewritten subquery.  This additional logic intercepts NULL values coming from the outer join and replaces them with the value that the subquery's expression would return on empty input.
    
    ## How was this patch tested?
    Added regression tests to cover all branches of the updated rule (see changes to `SubquerySuite.scala`).
    Ran all existing automated regression tests after merging with latest trunk.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/frreiss/spark-sandbox master

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/13155.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #13155
    
----
commit 3b1649105869c72ccb16f86732e04829aaae0e93
Author: frreiss <fr...@us.ibm.com>
Date:   2016-05-16T17:58:00Z

    Commit before merge.

commit 58df60d5468e53c4b6fc41a1d7c896abfb01cdd1
Author: frreiss <fr...@us.ibm.com>
Date:   2016-05-16T17:58:21Z

    Merge branch 'master' of https://github.com/apache/spark

commit 910cbf54e2300a57640e017610c204da2d462964
Author: frreiss <fr...@us.ibm.com>
Date:   2016-05-16T20:46:55Z

    Merge branch 'master' of https://github.com/apache/spark

commit 76d9f4528b8536d1e5680279ab76b9e26dd3a873
Author: frreiss <fr...@us.ibm.com>
Date:   2016-05-17T14:52:46Z

    Merge branch 'master' of https://github.com/apache/spark

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64834409
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    --- End diff --
    
    Another option could be update the expressions in outer query to use the new exprId (by keep these mapping as an hash map).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63633509
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala ---
    @@ -293,4 +293,65 @@ class SubquerySuite extends QueryTest with SharedSQLContext {
             """.stripMargin),
           Row(3) :: Nil)
       }
    +
    +  test("COUNT bug in WHERE clause (Filter)") {
    --- End diff --
    
    can you add the jira ticket number to the test case names?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64835067
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    --- End diff --
    
    style: use a variable for `isFiltered `?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-219902208
  
    cc @hvanhovell @davies 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63798715
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    --- End diff --
    
    Update: The relevant upstream checks are in CheckAnalysis.checkAnalysis():
    ```scala
    case s @ ScalarSubquery(query, conditions, _) if conditions.nonEmpty =>
        ...
        def cleanQuery(p: LogicalPlan): LogicalPlan = p match {
          case SubqueryAlias(_, child) => cleanQuery(child)
          case Project(_, child) => cleanQuery(child)
          case child => child
        }
    
        cleanQuery(query) match {
          case a: Aggregate => checkAggregate(a)
          case Filter(_, a: Aggregate) => checkAggregate(a)
          case fail => failAnalysis(s"Correlated scalar subqueries must be Aggregated: $fail")
        }
    ```
    These checks will accept a chain of operators above the Aggregate that match
    the regex:
        (SubqueryAlias|Project)*(Filter)?
    
    It doesn't look like AstBuilder will generate a SubqueryAlias nested immediately inside a SubqueryAlias, but all other matches of the above regex are possible.
    
    I'll update `evalOnZeroTups()` above to rewrite combinations of Project, Filter, and Aggregate into an expression, then statically evaluate the expression.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63767862
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala ---
    @@ -293,4 +293,65 @@ class SubquerySuite extends QueryTest with SharedSQLContext {
             """.stripMargin),
           Row(3) :: Nil)
       }
    +
    +  test("COUNT bug in WHERE clause (Filter)") {
    +
    +    // Case 1: Canonical example of the COUNT bug
    +    checkAnswer(
    +      sql("select l.a from l where (select count(*) from r where l.a = r.c) < l.a"),
    +      Row(1) :: Row(1) :: Row(3) :: Row(6) :: Nil)
    +
    +    // Case 2: count(*) = 0; could be rewritten to NOT EXISTS but currently uses
    +    // a rewrite that is vulnerable to the COUNT bug
    +    checkAnswer(
    +      sql("select l.a from l where (select count(*) from r where l.a = r.c) = 0"),
    +      Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil)
    +
    +    // Case 3: COUNT bug without a COUNT aggregate
    +    checkAnswer(
    +      sql("select l.a from l where (select sum(r.d) is null from r where l.a = r.c)"),
    +      Row(1) :: Row(1) ::Row(null) :: Row(null) :: Row(6) :: Nil)
    +
    --- End diff --
    
    Will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63759450
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    --- End diff --
    
    Interesting. I had thought that the checks in the Analyzer would ensure that only Aggregate nodes could appear immediately under a ScalarSubquery node. Obviously that is not the case.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65097239
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    --- End diff --
    
    Use an ArrayBuffer here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-222022339
  
    @frreiss Thanks for working on this. Had left some comments on how to rewrite the subquery, let me know how do you think, thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65093493
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    --- End diff --
    
    Nit: `Map.empty`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64833603
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    +          val renumberedQuery = query transformExpressions {
    +            case a@Alias(c, n) if a.exprId == origOutput.exprId => Alias(c, n)(exprId = newExprId)
    +          }
    +
    +          val filteredId = NamedExpression.newExprId
    +          val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +          val withIsFiltered = addIsFiltered(renumberedQuery, filteredId)
    +          val aggValRef = renumberedQuery.output.head
    +
    +          // CASE WHEN isFiltered IS NULL THEN COALESCE(aggVal, resultOnZeroTups)
    +          //      WHEN isFiltered THEN CAST(null AS <type of aggVal>)
    +          //      ELSE aggVal END
    +          val caseExpr = Alias(CaseWhen(
    +            Seq((IsNull(isFilteredRef), Coalesce(Seq(aggValRef,
    +              Literal(resultWithZeroTups.getOrElse(null))))),
    --- End diff --
    
    resultWithZeroTups can't be empty here, so it's better to use `resultWithZeroTups.get`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65096856
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    --- End diff --
    
    I think we also need to `null` out `AttributeReference`s. This is a (contrived) example when we need this:
    ```scala
    range(0, 15).createOrReplaceTempView("a")
    range(0, 100).select((rand() * 10).cast("int").as("key"), $"id".as("value")).createOrReplaceTempView("b")
    sql(
    """
    select  a.id,
            (select b.key + count(*) from b where b.key = a.id) cnt
    from    a
    """)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-221341883
  
    **[Test build #3014 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3014/consoleFull)** for PR 13155 at commit [`1b4ba5e`](https://github.com/apache/spark/commit/1b4ba5ed629d9b1e72d919d89b3592f7b29f3f3c).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63659398
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    --- End diff --
    
    Why do we need to use an `Alias` here? We are only evaluating the expression, and are not binding to the result of another operator.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubque...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on the issue:

    https://github.com/apache/spark/pull/13155
  
    LGTM - merging to master/2.0 (will do a small follow-up)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64941953
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    --- End diff --
    
    Fixed that in my local copy. Will include the change in the next update.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63933593
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    --- End diff --
    
    Upon further reflection, a Filter node above the Aggregate creates additional issues. If you rewrite the subquery into an outer join above the Filter, it may be impossible to distinguish between two cases:
    - The original subquery returns null because an aggregate value did not pass the HAVING clause
    - The aggregate in the original subquery is evaluated over zero tuples
    
    In both cases, the outer join will return a tuple containing nulls.
    For example, the query:
    ```sql
    select * from l where (select count(*) cnt from r where l.a = r.c having cnt = 0) = 0
    ```
    would turn into
    ```sql
    select * 
    from l left outer join (select c, count(*) cnt from r group by c having cnt = 0) sq
        on l.a = sq.c 
    where sq.cnt = 0
    ```
    Note how the rewritten subquery `select c, count(*) cnt from r group by c having cnt = 0` always returns zero tuples.
    
    I'm working through a few possible solutions, trying to see if one of them is guaranteed to be correct.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66680599
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1696,205 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr: Expression, bindings: Map[ExprId, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr: Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    // Also replace attribute refs (for example, for grouping columns) with NULL.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +
    +      case AttributeReference(_, _, _, _) => Literal.default(NullType)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = {
    --- End diff --
    
    style: no space before colon


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64961859
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    +          val renumberedQuery = query transformExpressions {
    +            case a@Alias(c, n) if a.exprId == origOutput.exprId => Alias(c, n)(exprId = newExprId)
    +          }
    +
    +          val filteredId = NamedExpression.newExprId
    +          val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +          val withIsFiltered = addIsFiltered(renumberedQuery, filteredId)
    --- End diff --
    
    I think we could keep the columns that is used by the predicate. Above Aggregate, there should be only Project and Filter (optional), should be easy, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-219855556
  
    Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63736050
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    --- End diff --
    
    For example, the (somewhat contrived) query:
    ```scala
    range(0, 10).registerTempTable("a")
    val plan = sql("select * from a where (select count(*) x from a aa where aa.id = a.id and aa.id % 2 = 0 having x > 0) > 0")
    plan.explain(true)
    ```
    results in the following analyzed plan:
    ```
    == Analyzed Logical Plan ==
    id: bigint
    Project [id#0L]
    +- Filter (scalar-subquery#27 [(id#0L#41L = id#0L)] > cast(0 as bigint))
       :  +- SubqueryAlias scalar-subquery#27 [(id#0L#41L = id#0L)]
       :     +- Project [x#26L,id#0L AS id#0L#41L]
       :        +- Filter (x#26L > cast(0 as bigint))
       :           +- Aggregate [id#0L], [(count(1),mode=Complete,isDistinct=false) AS x#26L,id#0L]
       :              +- Filter ((id#0L % cast(2 as bigint)) = cast(0 as bigint))
       :                 +- SubqueryAlias aa
       :                    +- SubqueryAlias a
       :                       +- Range 0, 10, 1, 8, [id#0L]
       +- SubqueryAlias a
          +- Range 0, 10, 1, 8, [id#0L]
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64834162
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    --- End diff --
    
    It's a little bit risk to replace all the exprId with new one, the exprId could be cached somewhere without replacing.
    
    Maybe we just use the same exprId for new Alias. It's a little bit confusing, but we already do this in a few places (change the nullability of a column without creating new one).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63945219
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    --- End diff --
    
    We could move the filter into the `left outer join` condition; this is not trivial but might work.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-219952769
  
    **[Test build #2990 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2990/consoleFull)** for PR 13155 at commit [`76d9f45`](https://github.com/apache/spark/commit/76d9f4528b8536d1e5680279ab76b9e26dd3a873).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66682052
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1696,205 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr: Expression, bindings: Map[ExprId, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr: Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    // Also replace attribute refs (for example, for grouping columns) with NULL.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +
    +      case AttributeReference(_, _, _, _) => Literal.default(NullType)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map.empty
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId, None)
    +            case _ => (ne.exprId, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the innermost query block
    +   * (first part of returned value), the HAVING clause of the innermost query block
    +   * (optional second part) and the parts below the HAVING CLAUSE (third part).
    +   */
    +  private def splitSubquery(plan: LogicalPlan) : (Seq[LogicalPlan], Option[Filter], Aggregate) = {
    +    val topPart = ArrayBuffer.empty[LogicalPlan]
    +    var bottomPart : LogicalPlan = plan
    +    while (true) {
    +      bottomPart match {
    +        case havingPart@Filter(_, aggPart@Aggregate(_, _, _)) =>
    +          return (topPart, Option(havingPart), aggPart.asInstanceOf[Aggregate])
    +
    +        case aggPart@Aggregate(_, _, _) =>
    +          // No HAVING clause
    +          return (topPart, None, aggPart)
    +
    +        case p@Project(_, child) =>
    +          topPart += p
    +          bottomPart = child
    +
    +        case s@SubqueryAlias(_, child) =>
    +          topPart += s
    +          bottomPart = child
    +
    +        case Filter(_, op@_) =>
    +          sys.error(s"Correlated subquery has unexpected operator $op below filter")
    +
    +        case op@_ => sys.error(s"Unexpected operator $op in correlated subquery")
    +      }
    +    }
    +
    +    sys.error("This line should be unreachable")
    +  }
    +
    +
    +
    +  // Name of generated column used in rewrite below
    +  val ALWAYS_TRUE_COLNAME = "alwaysTrue"
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          // CASE 1: Subquery guaranteed not to have the COUNT bug
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Subquery might have the COUNT bug. Add appropriate corrections.
    +          val (topPart, havingNode, aggNode) = splitSubquery(query)
    +
    +          // The next two cases add a leading column to the outer join input to make it
    +          // possible to distinguish between the case when no tuples join and the case
    +          // when the tuple that joins contains null values.
    +          // The leading column always has the value TRUE.
    +          val alwaysTrueExprId = NamedExpression.newExprId
    +          val alwaysTrueExpr = Alias(Literal.TrueLiteral,
    +            ALWAYS_TRUE_COLNAME)(exprId = alwaysTrueExprId)
    +          val alwaysTrueRef = AttributeReference(ALWAYS_TRUE_COLNAME,
    +            BooleanType)(exprId = alwaysTrueExprId)
    +
    +          val aggValRef = query.output.head
    +
    +          if (!havingNode.isDefined) {
    +            // CASE 2: Subquery with no HAVING clause
    +            Project(
    +              currentChild.output :+
    +                Alias(
    +                  If(IsNull(alwaysTrueRef),
    +                    Literal(resultWithZeroTups.get, origOutput.dataType),
    +                    aggValRef), origOutput.name)(exprId = origOutput.exprId),
    +              Join(currentChild,
    +                Project(query.output :+ alwaysTrueExpr, query),
    +                LeftOuter, conditions.reduceOption(And)))
    +
    +          } else {
    +            // CASE 3: Subquery with HAVING clause. Pull the HAVING clause above the join.
    +            // Need to modify any operators below the join to pass through all columns
    +            // referenced in the HAVING clause.
    +            var subqueryRoot : UnaryNode = aggNode
    +            val havingInputs : Seq[NamedExpression] = aggNode.output
    +
    +            topPart.reverse.foreach(
    --- End diff --
    
    style. do the pattern match in the foreach, i.e.:
    ```scala
    topPart.reverse.foreach {
      case Project(projList, _) =>
      ...
    }
    ```
    
    Also @'s


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64836855
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    +          val renumberedQuery = query transformExpressions {
    +            case a@Alias(c, n) if a.exprId == origOutput.exprId => Alias(c, n)(exprId = newExprId)
    +          }
    +
    +          val filteredId = NamedExpression.newExprId
    +          val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +          val withIsFiltered = addIsFiltered(renumberedQuery, filteredId)
    --- End diff --
    
    Instead of turn the predicates in subquery into a projection, can we pull out the predicate in HAVING, then evaluate that after the join, so the result column will be.
    ```
    Project(currentChild.output :+ If(predicate, agg, Literal(null)),
      Project(currentChild.output :+ Alias(Coalesce(Seq(aggvalRef, resultWithZeroTups)), "agg")())
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63786152
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    +        val origOutput = query.output.head
    +
    +        // Ensure the rewritten subquery returns the same result when a tuple from the
    +        // outer query block does not join with the subquery block.
    +        // val (outputExpr, rewrittenQuery) = aggFunc.defaultResult match {
    +        val (outputExpr, rewrittenQuery) = evalOnZeroTups(aggOutputExpr) match {
    +          case Some(value) =>
    +            val origExprId = origOutput.exprId
    +            val newExprId = NamedExpression.newExprId
    +
    +            // Renumber the original output, because the outer query refers to its ID.
    +            val newQuery = query transformExpressions {
    +              case Alias(c, n) => Alias(c, n)(exprId = newExprId)
    --- End diff --
    
    Oops, there should be an "if" after that "case" expression. I'll update that expression to only renumber `origExprId` to `newExprId`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64995985
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    +          val renumberedQuery = query transformExpressions {
    +            case a@Alias(c, n) if a.exprId == origOutput.exprId => Alias(c, n)(exprId = newExprId)
    +          }
    +
    +          val filteredId = NamedExpression.newExprId
    +          val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +          val withIsFiltered = addIsFiltered(renumberedQuery, filteredId)
    --- End diff --
    
    Sure, I can modify the rewrite there to pull the rewritten filter above the join.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65583548
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    --- End diff --
    
    The test on the next line doesn't compile without the cast. The condition in a Filter node is of type Expression, and `Expresssion.eval()` returns `Any`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66561017
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    --- End diff --
    
    Yes, that is true. Added a test case and an additional clause in that case statement in my local copy.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64835290
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    --- End diff --
    
    ```
    if () {
      var rewrittenQuery: LogicalPlan = filterAsProject
       while() {
       }
      rewrittenQuery
    } else {
      Project(xx, subqueryPlan)
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubque...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on the issue:

    https://github.com/apache/spark/pull/13155
  
    Updated changes are in. Running a full regression suite overnight.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66564793
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    --- End diff --
    
    @hvanhovell the conditions for the COUNT bug in this rule are actually a bit less strict. If the correlated subquery returns an answer other than NULL when the correlation bindings do not join, then it may be possible to have the COUNT bug. Take this query for example:
    ```sql
    select l.a from l where
            (select case when count(*) = 1 then null else count(*) end as cnt
            from r where l.a = r.c) = 0
    ```
    This subquery returns NULL when exactly 1 tuple from `r` joins with a set of correlation bindings and a non-NULL value otherwise. Rewrite this query without compensating for the COUNT bug and you get:
    ```sql
    select l.a from l left join 
      (select r.c, case when count(*) = 1 then null else count(*) end as cnt from r group by r.c) SQ
    on l.a = SQ.c
    where SQ.cnt = 0
    ```
    If the value of `l.a` matches zero tuples in `r`, the outer join produces a tuple with `cnt` set to NULL and the rewritten query incorrectly returns an empty result.
    
    Technically the full condition for the COUNT bug is that the subquery returns a non-NULL value when zero tuples join; *and* the outer query block will return a different answer if it receives that value instead of NULL. Checking the outer query block would be a somewhat-useful performance optimization, but it would require additional static evaluation code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65097053
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    --- End diff --
    
    We could just call `evalAggOnZeroTups` on every expression. That would also allow for more complex groupBy expressions (currently not needed though).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubque...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the issue:

    https://github.com/apache/spark/pull/13155
  
    LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63730352
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    --- End diff --
    
    I am also not sure that you should assume that an aggregate is returned.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66509405
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    --- End diff --
    
    Fixed in my local copy


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-221336991
  
    Could one of the committers please trigger another build on this PR? The change set passes all the tests on my machine, but it's good to be safe.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65093043
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    --- End diff --
    
    Why do you use `ExprId.id` instead of entire `ExprId` as map keys?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64996012
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    --- End diff --
    
    I originally tried using the same exprID in the new Alias, but a downstream rewrite kept going into infinite recursion. Not sure which one -- the stack kept blowing up before the trace said which rule was running. I figured that multiple uses of the same exprID across operators was not supported. If that approach is supposed to work, I can track down the rule that was blowing up and open another defect if you'd like.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-221365033
  
    **[Test build #3014 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3014/consoleFull)** for PR 13155 at commit [`1b4ba5e`](https://github.com/apache/spark/commit/1b4ba5ed629d9b1e72d919d89b3592f7b29f3f3c).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63756577
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    --- End diff --
    
    Works for me. I'll change that expression to getOrElse().


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63651943
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    --- End diff --
    
    Nit: space


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66681873
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1696,205 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr: Expression, bindings: Map[ExprId, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr: Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    // Also replace attribute refs (for example, for grouping columns) with NULL.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +
    +      case AttributeReference(_, _, _, _) => Literal.default(NullType)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map.empty
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId, None)
    +            case _ => (ne.exprId, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the innermost query block
    +   * (first part of returned value), the HAVING clause of the innermost query block
    +   * (optional second part) and the parts below the HAVING CLAUSE (third part).
    +   */
    +  private def splitSubquery(plan: LogicalPlan) : (Seq[LogicalPlan], Option[Filter], Aggregate) = {
    +    val topPart = ArrayBuffer.empty[LogicalPlan]
    +    var bottomPart : LogicalPlan = plan
    +    while (true) {
    +      bottomPart match {
    +        case havingPart@Filter(_, aggPart@Aggregate(_, _, _)) =>
    +          return (topPart, Option(havingPart), aggPart.asInstanceOf[Aggregate])
    +
    +        case aggPart@Aggregate(_, _, _) =>
    +          // No HAVING clause
    +          return (topPart, None, aggPart)
    +
    +        case p@Project(_, child) =>
    +          topPart += p
    +          bottomPart = child
    +
    +        case s@SubqueryAlias(_, child) =>
    +          topPart += s
    +          bottomPart = child
    +
    +        case Filter(_, op@_) =>
    +          sys.error(s"Correlated subquery has unexpected operator $op below filter")
    +
    +        case op@_ => sys.error(s"Unexpected operator $op in correlated subquery")
    +      }
    +    }
    +
    +    sys.error("This line should be unreachable")
    +  }
    +
    +
    +
    +  // Name of generated column used in rewrite below
    +  val ALWAYS_TRUE_COLNAME = "alwaysTrue"
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          // CASE 1: Subquery guaranteed not to have the COUNT bug
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Subquery might have the COUNT bug. Add appropriate corrections.
    +          val (topPart, havingNode, aggNode) = splitSubquery(query)
    +
    +          // The next two cases add a leading column to the outer join input to make it
    +          // possible to distinguish between the case when no tuples join and the case
    +          // when the tuple that joins contains null values.
    +          // The leading column always has the value TRUE.
    +          val alwaysTrueExprId = NamedExpression.newExprId
    +          val alwaysTrueExpr = Alias(Literal.TrueLiteral,
    +            ALWAYS_TRUE_COLNAME)(exprId = alwaysTrueExprId)
    +          val alwaysTrueRef = AttributeReference(ALWAYS_TRUE_COLNAME,
    +            BooleanType)(exprId = alwaysTrueExprId)
    +
    +          val aggValRef = query.output.head
    +
    +          if (!havingNode.isDefined) {
    +            // CASE 2: Subquery with no HAVING clause
    +            Project(
    +              currentChild.output :+
    +                Alias(
    +                  If(IsNull(alwaysTrueRef),
    +                    Literal(resultWithZeroTups.get, origOutput.dataType),
    +                    aggValRef), origOutput.name)(exprId = origOutput.exprId),
    +              Join(currentChild,
    +                Project(query.output :+ alwaysTrueExpr, query),
    +                LeftOuter, conditions.reduceOption(And)))
    +
    +          } else {
    +            // CASE 3: Subquery with HAVING clause. Pull the HAVING clause above the join.
    +            // Need to modify any operators below the join to pass through all columns
    +            // referenced in the HAVING clause.
    +            var subqueryRoot : UnaryNode = aggNode
    --- End diff --
    
    Style colon. 
    same for line below


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65454461
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    --- End diff --
    
    Fixed in my local copy.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66560947
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    --- End diff --
    
    This code no longer exists in my local copy after the refactoring below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubque...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on the pull request:

    https://github.com/apache/spark/pull/13155
  
    Thanks @hvanhovell for the additional pass of review! I'll be preparing my slides for Spark Summit all day today but will come back to this PR as soon as that's done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64834697
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    --- End diff --
    
    Why create a new one, but not passing the `isFilteredRef ` here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63626503
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala ---
    @@ -293,4 +293,65 @@ class SubquerySuite extends QueryTest with SharedSQLContext {
             """.stripMargin),
           Row(3) :: Nil)
       }
    +
    +  test("COUNT bug in WHERE clause (Filter)") {
    +
    +    // Case 1: Canonical example of the COUNT bug
    +    checkAnswer(
    +      sql("select l.a from l where (select count(*) from r where l.a = r.c) < l.a"),
    +      Row(1) :: Row(1) :: Row(3) :: Row(6) :: Nil)
    +
    +    // Case 2: count(*) = 0; could be rewritten to NOT EXISTS but currently uses
    +    // a rewrite that is vulnerable to the COUNT bug
    +    checkAnswer(
    +      sql("select l.a from l where (select count(*) from r where l.a = r.c) = 0"),
    +      Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil)
    +
    +    // Case 3: COUNT bug without a COUNT aggregate
    +    checkAnswer(
    +      sql("select l.a from l where (select sum(r.d) is null from r where l.a = r.c)"),
    +      Row(1) :: Row(1) ::Row(null) :: Row(null) :: Row(6) :: Nil)
    +
    +  }
    +
    +  test("COUNT bug in SELECT clause (Project)") {
    +    checkAnswer(
    +      sql("select a, (select count(*) from r where l.a = r.c) as cnt from l"),
    +      Row(1, 0) :: Row(1, 0) :: Row(2, 2) :: Row(2, 2) :: Row(3, 1) :: Row(null, 0)
    +        :: Row(null, 0) :: Row(6, 1) :: Nil)
    +  }
    +
    +  test("COUNT bug in HAVING clause (Filter)") {
    +    checkAnswer(
    +      sql("select l.a as grp_a from l group by l.a " +
    +        "having (select count(*) from r where grp_a = r.c) = 0 " +
    +        "order by grp_a"),
    +      Row(null) :: Row(1) :: Nil)
    +  }
    +
    +  test("COUNT bug in Aggregate") {
    +    checkAnswer(
    +      sql("select l.a as aval, sum((select count(*) from r where l.a = r.c)) as cnt " +
    +        "from l group by l.a order by aval"),
    +      Row(null, 0) :: Row(1, 0) :: Row(2, 4) :: Row(3, 1) :: Row(6, 1)  :: Nil)
    +  }
    +
    +  test("COUNT bug negative examples") {
    +    // Case 1: Potential COUNT bug case that was working correctly prior to the fix
    +    checkAnswer(
    +      sql("select l.a from l where (select sum(r.d) from r where l.a = r.c) is null"),
    +      Row(1) :: Row(1) :: Row(null) :: Row(null) :: Row(6) :: Nil)
    +
    +    // Case 2: COUNT aggregate but no COUNT bug due to > 0 test.
    +    checkAnswer(
    +      sql("select l.a from l where (select count(*) from r where l.a = r.c) > 0"),
    +      Row(2) :: Row(2) :: Row(3) :: Row(6) :: Nil)
    +
    +    // Case 3: COUNT inside aggregate expression but no COUNT bug.
    +    checkAnswer(
    +      sql("select l.a from l where (select count(*) + sum(r.d) from r where l.a = r.c) = 0"),
    +      Nil)
    +
    +
    --- End diff --
    
    (Maybe here too..)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66558125
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    --- End diff --
    
    Fixed in my local copy.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66680913
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1696,205 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr: Expression, bindings: Map[ExprId, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr: Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    // Also replace attribute refs (for example, for grouping columns) with NULL.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +
    +      case AttributeReference(_, _, _, _) => Literal.default(NullType)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map.empty
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId, None)
    +            case _ => (ne.exprId, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the innermost query block
    +   * (first part of returned value), the HAVING clause of the innermost query block
    +   * (optional second part) and the parts below the HAVING CLAUSE (third part).
    +   */
    +  private def splitSubquery(plan: LogicalPlan) : (Seq[LogicalPlan], Option[Filter], Aggregate) = {
    +    val topPart = ArrayBuffer.empty[LogicalPlan]
    +    var bottomPart : LogicalPlan = plan
    --- End diff --
    
    style: no space before colon


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64832620
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    +          val renumberedQuery = query transformExpressions {
    +            case a@Alias(c, n) if a.exprId == origOutput.exprId => Alias(c, n)(exprId = newExprId)
    --- End diff --
    
    Is it possible that there is origOutput.exprId come from an AttributeReference? for example:
    ```
    SELECT (SELECT gid FROM g where gid=t.id) FROM t
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63954767
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    --- End diff --
    
    I came across an additional counterexample just now:
    ```sql
    select l.a from l where
            (select case when count(*) = 1 then null else count(*) from r where l.a = r.c) = 0
    ```
    This subquery returns null when count(*) on the inner query block is 1. The rewrite in this PR turns the overall query into:
    ```sql
    select * 
    from l left outer join 
        (select c, case when count(*) = 1 then null else count(*) as cnt from r group by c) sq
        on l.a = sq.c 
    where coalesce(sq.cnt, 0) = 0
    ```
    This result is incorrect; if exactly one tuple from R joins with a tuple of L, the query will return the L tuple, even though that tuple should not be returned.
    
    So, to summarize, the operators above the outer join need to be able to discern between *four* different cases:
    1. A tuple from the outer query block joins with one or more tuples in the subquery and produces an aggregate result that is *not* null
    2. A tuple from the outer query block joins with one or more tuples in the subquery and produces an aggregate result that *is* null
    3. A tuple from the outer query block does not join with any tuples in the subquery
    4. A tuple from the outer query block joins with tuples in the subquery and produces an aggregate result that is filtered out by a HAVING clause
    
    A single result column is simply not able to encode all four cases (or even the first three). The output of the outer join needs to have at least two columns to pass through enough information. 
    Here's a scheme that will work as far as I can see: The result of the outer join contains the correlation columns, plus a column `aggVal` for the aggregate value, plus an additional column `isFiltered`. The four cases above are encoded as follows:
    1. `aggVal is not null and isFiltered = false`
    2. `aggVal is null and isFiltered = false`
    3. `isFiltered is null`
    4. `isFiltered = true`
    
    Then the coalesce expression above the outer join turns into a CASE statement:
    ```sql
    case 
       when isFiltered is null then coalesce(aggVal, <SQ return value on empty input>)
       when isFiltered then null
       else aggVal
    ```
    
    Here's pseudocode for a rewrite that should produce the correct values of `isFiltered` and `aggVal`:
    ```
    if (subquery returns null when zero tuples join) 
    then
      Use the original rewrite.
    else 
      if (subquery has a Filter above the Aggregate node)
      then
        Replace the Filter node with a Project that computes the value of isFiltered.
        Rewrite nodes above the Filter node so they pass through the isFiltered column.
      else
         Add an isFiltered column with a hard-coded value of false to the top Project in the subquery.
      endif
      Create a left outer join between the outer query block and the rewritten subquery.
      Put the case statement in the previous listing into the Project operator above the outer join.
    endif
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64944724
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    +          val renumberedQuery = query transformExpressions {
    +            case a@Alias(c, n) if a.exprId == origOutput.exprId => Alias(c, n)(exprId = newExprId)
    +          }
    +
    +          val filteredId = NamedExpression.newExprId
    +          val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +          val withIsFiltered = addIsFiltered(renumberedQuery, filteredId)
    --- End diff --
    
    I thought about pulling the filter expression up, but projections above the innermost clause of the subquery can modify the values that the HAVING clause references. For example, the subquery could be something like:
    ```
    (select cnt + 1 from (select count(*) as cnt from r where r.a = s.b having cnt >= s.c))
    ```
    To handle these cases correctly, we would need to rewrite all of the projection expressions to pass through any columns used by the HAVING clause; either that or selectively pull up the HAVING clause based on a static analysis of the projections above the innermost correlated subquery.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65102803
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    +          val renumberedQuery = query transformExpressions {
    +            case a@Alias(c, n) if a.exprId == origOutput.exprId => Alias(c, n)(exprId = newExprId)
    +          }
    +
    +          val filteredId = NamedExpression.newExprId
    +          val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +          val withIsFiltered = addIsFiltered(renumberedQuery, filteredId)
    --- End diff --
    
    @frreiss we already pull-out correlated predicates in the Analyzer. Adding these predicates might simplify this rule.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65093449
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    --- End diff --
    
    Do you need the type cast?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65103010
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    --- End diff --
    
    It might be easier to check if the first column is `nullable`. Then we only need to do the `evalSubqueryOnZeroTups` eval if we need to, and then we might have an opportunity to simplify the code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubque...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on the issue:

    https://github.com/apache/spark/pull/13155
  
    Tests ran successfully on my machine.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63757089
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    --- End diff --
    
    The Alias was paranoia on my part. I noticed that AggregateExpression nodes had exprIds and I wasn't sure if those IDs could be consumed somewhere else in the expression evaluation code, so I added the Alias to retain the original IDs. All my test cases work without that additional Alias. I'll remove the Alias node.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-221419331
  
    @frreiss this looks promising. I will take a more in depth look at it tomorrow evening (GMT).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-219936663
  
    **[Test build #2990 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2990/consoleFull)** for PR 13155 at commit [`76d9f45`](https://github.com/apache/spark/commit/76d9f4528b8536d1e5680279ab76b9e26dd3a873).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66680885
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1696,205 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr: Expression, bindings: Map[ExprId, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr: Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    // Also replace attribute refs (for example, for grouping columns) with NULL.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +
    +      case AttributeReference(_, _, _, _) => Literal.default(NullType)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map.empty
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId, None)
    --- End diff --
    
    style: `case Alias(_: AttributeReference, _) =>`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66561815
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    --- End diff --
    
    Update: The infinite recursion was due to two instances of using `transform` instead of `transformUp` when replacing Alias nodes with their referents. Fixed those in my local copy and got rid of the ID renumbering in the rule.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-220086644
  
    @frreiss thanks for working on this. The fix you are proposing is the right one. There are still a few cornercases which need to be addressed before we can go ahead and merge this; see the comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66558119
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    --- End diff --
    
    Fixed in my local copy.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66478261
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    --- End diff --
    
    No particular reason for using `ExprId.id`. I'll change to using the entire class as a key.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66682369
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1696,205 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr: Expression, bindings: Map[ExprId, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr: Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    // Also replace attribute refs (for example, for grouping columns) with NULL.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +
    +      case AttributeReference(_, _, _, _) => Literal.default(NullType)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map.empty
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId, None)
    +            case _ => (ne.exprId, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the innermost query block
    +   * (first part of returned value), the HAVING clause of the innermost query block
    +   * (optional second part) and the parts below the HAVING CLAUSE (third part).
    +   */
    +  private def splitSubquery(plan: LogicalPlan) : (Seq[LogicalPlan], Option[Filter], Aggregate) = {
    +    val topPart = ArrayBuffer.empty[LogicalPlan]
    +    var bottomPart : LogicalPlan = plan
    +    while (true) {
    +      bottomPart match {
    +        case havingPart@Filter(_, aggPart@Aggregate(_, _, _)) =>
    +          return (topPart, Option(havingPart), aggPart.asInstanceOf[Aggregate])
    +
    +        case aggPart@Aggregate(_, _, _) =>
    +          // No HAVING clause
    +          return (topPart, None, aggPart)
    +
    +        case p@Project(_, child) =>
    +          topPart += p
    +          bottomPart = child
    +
    +        case s@SubqueryAlias(_, child) =>
    +          topPart += s
    +          bottomPart = child
    +
    +        case Filter(_, op@_) =>
    +          sys.error(s"Correlated subquery has unexpected operator $op below filter")
    +
    +        case op@_ => sys.error(s"Unexpected operator $op in correlated subquery")
    +      }
    +    }
    +
    +    sys.error("This line should be unreachable")
    +  }
    +
    +
    +
    +  // Name of generated column used in rewrite below
    +  val ALWAYS_TRUE_COLNAME = "alwaysTrue"
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          // CASE 1: Subquery guaranteed not to have the COUNT bug
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Subquery might have the COUNT bug. Add appropriate corrections.
    +          val (topPart, havingNode, aggNode) = splitSubquery(query)
    +
    +          // The next two cases add a leading column to the outer join input to make it
    +          // possible to distinguish between the case when no tuples join and the case
    +          // when the tuple that joins contains null values.
    +          // The leading column always has the value TRUE.
    +          val alwaysTrueExprId = NamedExpression.newExprId
    +          val alwaysTrueExpr = Alias(Literal.TrueLiteral,
    +            ALWAYS_TRUE_COLNAME)(exprId = alwaysTrueExprId)
    +          val alwaysTrueRef = AttributeReference(ALWAYS_TRUE_COLNAME,
    +            BooleanType)(exprId = alwaysTrueExprId)
    +
    +          val aggValRef = query.output.head
    +
    +          if (!havingNode.isDefined) {
    +            // CASE 2: Subquery with no HAVING clause
    +            Project(
    +              currentChild.output :+
    +                Alias(
    +                  If(IsNull(alwaysTrueRef),
    +                    Literal(resultWithZeroTups.get, origOutput.dataType),
    +                    aggValRef), origOutput.name)(exprId = origOutput.exprId),
    +              Join(currentChild,
    +                Project(query.output :+ alwaysTrueExpr, query),
    +                LeftOuter, conditions.reduceOption(And)))
    +
    +          } else {
    +            // CASE 3: Subquery with HAVING clause. Pull the HAVING clause above the join.
    +            // Need to modify any operators below the join to pass through all columns
    +            // referenced in the HAVING clause.
    +            var subqueryRoot : UnaryNode = aggNode
    +            val havingInputs : Seq[NamedExpression] = aggNode.output
    +
    +            topPart.reverse.foreach(
    +              _ match {
    +                case Project(projList, _) =>
    +                  subqueryRoot = Project(projList ++ havingInputs, subqueryRoot)
    +                case s@SubqueryAlias(alias, _) => subqueryRoot = SubqueryAlias(alias, subqueryRoot)
    +                case op@_ => sys.error(s"Unexpected operator $op in corelated subquery")
    +              }
    +            )
    +
    +            // CASE WHEN alwayTrue IS NULL THEN resultOnZeroTups
    +            //      WHEN NOT (original HAVING clause expr) THEN CAST(null AS <type of aggVal>)
    +            //      ELSE (aggregate value) END AS (original column name)
    +            val caseExpr = Alias(CaseWhen(
    +              Seq[(Expression, Expression)] (
    --- End diff --
    
    Do we need to type the Seq?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65093465
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    --- End diff --
    
    NIT: `bindings.isEmpty`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63770055
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    --- End diff --
    
    I can help you with that. The following construction 'rules' apply:
    
    1. A `ScalarSubquery` has to contain an `Aggregate`. The Aggregate is can be nested in a `Filter` when a `HAVING` clauses is specified. This is the basic `ScalarSubquery`.
    2. If the `ScalarSubquery` produces an `Attribute` that is also present in the outer query, then we wrap it with a `Project` and alias the offending attributes in order to prevent attribute clashes between the inner and outer queries.
    3. In case of SQL generation we can add further `Project` and `SubqueryAlias` nodes.
    
    The example applies rules 1 & 2.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66681029
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1696,205 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr: Expression, bindings: Map[ExprId, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr: Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    // Also replace attribute refs (for example, for grouping columns) with NULL.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +
    +      case AttributeReference(_, _, _, _) => Literal.default(NullType)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map.empty
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId, None)
    +            case _ => (ne.exprId, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the innermost query block
    +   * (first part of returned value), the HAVING clause of the innermost query block
    +   * (optional second part) and the parts below the HAVING CLAUSE (third part).
    +   */
    +  private def splitSubquery(plan: LogicalPlan) : (Seq[LogicalPlan], Option[Filter], Aggregate) = {
    +    val topPart = ArrayBuffer.empty[LogicalPlan]
    +    var bottomPart : LogicalPlan = plan
    +    while (true) {
    +      bottomPart match {
    +        case havingPart@Filter(_, aggPart@Aggregate(_, _, _)) =>
    --- End diff --
    
    style spaces between @ elements, i.e.: `havingPart @ Filter`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66509510
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    --- End diff --
    
    Fixed in my local copy.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66680557
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1696,205 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr: Expression, bindings: Map[ExprId, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr: Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    // Also replace attribute refs (for example, for grouping columns) with NULL.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +
    +      case AttributeReference(_, _, _, _) => Literal.default(NullType)
    --- End diff --
    
    style: `case _: AttributeReference => ...` 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64943404
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    --- End diff --
    
    Fixed in my copy.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubque...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/13155
  
    **[Test build #3074 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3074/consoleFull)** for PR 13155 at commit [`8cd2877`](https://github.com/apache/spark/commit/8cd2877179dded4557c8da92e5b16011637289b0).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubque...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the issue:

    https://github.com/apache/spark/pull/13155
  
    @hvanhovell Could you have a final look on this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubque...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on the issue:

    https://github.com/apache/spark/pull/13155
  
    @frreiss PR looks good. It does need a little bit of work in the style department.  I'll leave comments. Lemme know if you have time to address these; else I'll take over and do that chore (you'll still get the credit though).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63768232
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    --- End diff --
    
    I'm looking into exactly what types of plans can show up in a ScalarSubquery node that passes all the checks upstream of RewriteCorrelatedScalarSubquery. Should have an answer in a few hours.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66540105
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    --- End diff --
    
    @frreiss this bug only occurs when the result of the scalar subquery cannot be null. `COUNT(...)` being the prime example of this. We can identify this problem early and easily by checking the nullable flag. Am I missing something?
    
    BTW: Nice talk on SystemML!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66560868
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    +          val renumberedQuery = query transformExpressions {
    +            case a@Alias(c, n) if a.exprId == origOutput.exprId => Alias(c, n)(exprId = newExprId)
    --- End diff --
    
    I wasn't able to make the output of the subquery be an anonymous AttributeReference in my testing. The Analyzer adds aliases when the subquery's schema is anonymous.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65584546
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    --- End diff --
    
    The join attributes that the Analyzer adds to the Aggregate node are AttributeReference nodes, and the `evalAggOnZeroTups` method as currently written can't process them. A more comprehensive facility for statically evaluating expressions would be nice to have; but I'm hesitant to add such a mechanism as part of a bug fix. Perhaps a follow-on JIRA is in order?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66539170
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    --- End diff --
    
    Sorry, I'm having some trouble understanding this comment. I think you're suggesting that the rule should first check whether `query.output.head` (the output column of the scalar subquery) is a nullable column; then the rule should sometimes skip the call to `evalQueryOnZeroTups()` depending on the nullability of the column.
    
    I don't see how the check for nullability would allow one to skip the call to `evalQueryOnZeroTups()`. If the column is nullable, then the correlated subquery could return either null or a non-null value when zero tuples match; and the rule needs to call `evalQueryOnZeroTups` to distinguish between those two cases. If the column is not nullable, then the rule still needs to call `evalQueryOnZeroTups` to determine what non-null value the subquery returns when zero tuples match.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66680828
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1696,205 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr: Expression, bindings: Map[ExprId, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr: Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    // Also replace attribute refs (for example, for grouping columns) with NULL.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +
    +      case AttributeReference(_, _, _, _) => Literal.default(NullType)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map.empty
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.isEmpty) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    --- End diff --
    
    style: do the pattern match directly, i.e.:
    ```scala
    aggExpr.map {
      case _: AttributeReference =>
      case ...
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63651882
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    --- End diff --
    
    You can actually get the aggregate in the `case ...` statement


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64491862
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    --- End diff --
    
    no space before ':'


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubque...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/13155
  
    **[Test build #3074 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3074/consoleFull)** for PR 13155 at commit [`8cd2877`](https://github.com/apache/spark/commit/8cd2877179dded4557c8da92e5b16011637289b0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63730774
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    +        val origOutput = query.output.head
    +
    +        // Ensure the rewritten subquery returns the same result when a tuple from the
    +        // outer query block does not join with the subquery block.
    +        // val (outputExpr, rewrittenQuery) = aggFunc.defaultResult match {
    +        val (outputExpr, rewrittenQuery) = evalOnZeroTups(aggOutputExpr) match {
    +          case Some(value) =>
    +            val origExprId = origOutput.exprId
    +            val newExprId = NamedExpression.newExprId
    +
    +            // Renumber the original output, because the outer query refers to its ID.
    +            val newQuery = query transformExpressions {
    +              case Alias(c, n) => Alias(c, n)(exprId = newExprId)
    --- End diff --
    
    This will give every alias the same id! That is not what we want.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63651907
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    --- End diff --
    
    `getOrElse(...)?`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63823201
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1648,16 +1648,56 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        val resultLit = aggFunc.defaultResult match {
    +          case Some(lit) => lit
    +          case None => Literal.default(NullType)
    +        }
    +        Alias(resultLit, "aggVal") (exprId = resultId)
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    +        val aggOutputExpr = query.asInstanceOf[Aggregate].aggregateExpressions.head
    --- End diff --
    
    Sorry, didn't see your reply before I posted mine. I must not have refreshed my browser. Thanks for the info on the possible cases. I'm testing the updated static evaluation code now. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-220684859
  
    I've added additional changes to cover two additional cases that @hvanhovell pointed out on review, plus one additional case that came up while fixing the first two:
    - The correlated subquery may have a HAVING clause
    - The correlated subquery may be nested inside additional query blocks that apply projections
    - The correlated subquery may return NULL when the correlation bindings join
      with the subquery and a non-NULL value when bindings do *not* join, e.g. 
    ```sql
    select l.a from l where
      (select case when count(*) = 1 then null else count(*) end from r where l.a = r.c) = 0
    ```
    
    The logic for statically evaluating the subquery's aggregate expression now handles an Aggregate node with a chain of Filter and Project operators above it. The rewrite logic has an third case to handle subqueries with HAVING clauses. The second case from the original change set now adds an additional column to cover subqueries that return NULL with bindings that join the inner query block. I also addressed various other more minor review comments.
    
    After these changes, the algorithm for preventing the COUNT bug in scalar subqueries is now the following:
    ```
    V <- <value that the subquery returns when zero tuples join>
    if (V is null)
    then
      Use the original rewrite from SPARK-14785.
    else 
      if (subquery has a Filter above the Aggregate node)
      then
        Rewrite the Filter node to a Project that adds a Boolean column isFiltered.
        Rewrite nodes above the Filter node so they pass through the isFiltered column.
      else
        Add an isFiltered column with a hard-coded value of false to the top Project in the subquery.
      endif
      Create a left outer join between the outer query block and the rewritten subquery.
      Put the following case statement into the Project operator above the outer join.
          case 
           when isFiltered is null then coalesce(aggVal, V)
           when isFiltered then null
           else aggVal
    endif
    ```
    
    ### Correctness proof 
    
    Let *b* denote a tuple of correlation bindings from the outer query block.  Without loss of generality, assume that the correlated subquery has the plan `Project(Filter(Aggregate(Join({b},T))))`, where *T* is a table. Note that upstream checks in Catalyst ensure that the Aggregate node will produce exactly one tuple.
    
    Consider the evaluation of the original subquery on *b*. We can ask three questions about this evaluation:
    - Did *b* join with one or more tuples from *T*?
    - Did the Filter node reject the tuple that the Aggregate node returned?
    - Did the subquery return `null`?
    
    The answer to each of these questions must be "yes" or "no", leading to eight cases:
    
    Case \# | Empty join? | Agg filtered? | SQ returns null?
    ------ | ----------- | ------------- | ----------------
    1      | Yes         | Yes           | Yes
    2      | Yes         | Yes           | No
    3      | Yes         | No            | Yes
    4      | Yes         | No            | No
    5      | No          | Yes           | Yes
    6      | No          | Yes           | No
    7      | No          | No            | Yes
    8      | No          | No            | No
    
    Cases 2 and 6 are impossible. For the remaining cases, the rewritten subquery returns the correct result:
    - Case 1 and 3 => Subquery returns null on empty join result, so first branch of rewrite algorithm above applies. Outer join returns null for subquery result.
    - Case 4 => Subquery returns non-null answer on empty join result, so second or third branch of rewrite applies. Outer join in the rewritten query returns a tuple with `(aggVal, isFiltered)` set to `(null,null)`, so case statement at the top of the rewritten query returns the answer the subquery returns on an empty join, which is the correct answer.
    - Case 5 => If first branch of rewrite applies, the outer join in the rewritten query returns null. If second or third branch of rewrite applies, then outer join in rewritten query returns a tuple with `(aggVal, isFiltered)` set to `(null, true)`, so case statement at top of rewritten query returns null.
    - Case 7 => If first branch of rewrite applies, the outer join in the rewritten query returns null. For second or third branch of rewrite, outer join in rewritten query returns a tuple with `(aggVal, isFiltered)` set to `(null, false)`, so case statement at top of rewritten query returns null.
    - Case 8 => If first branch of rewrite applies, the outer join in the rewritten query returns null. For second or third branch of rewrite, outer join in rewritten query returns a tuple with `(aggVal, isFiltered)` set to `(<non-null value>, false)`, so case statement at top of rewritten query returns the non-null value of aggVal.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r66509444
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    --- End diff --
    
    Fixed in my local copy


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64942480
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    +    }
    +    return rewrittenQuery
    +  }
    +
    +  /**
        * Construct a new child plan by left joining the given subqueries to a base plan.
        */
       private def constructLeftJoins(
           child: LogicalPlan,
           subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = {
         subqueries.foldLeft(child) {
           case (currentChild, ScalarSubquery(query, conditions, _)) =>
    -        Project(
    -          currentChild.output :+ query.output.head,
    -          Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        val origOutput = query.output.head
    +
    +        val resultWithZeroTups = evalSubqueryOnZeroTups(query)
    +        if (resultWithZeroTups.isEmpty) {
    +          Project(
    +            currentChild.output :+ origOutput,
    +            Join(currentChild, query, LeftOuter, conditions.reduceOption(And)))
    +        } else {
    +          // Renumber the original output, because the outer query refers to its ID.
    +          val newExprId = NamedExpression.newExprId
    +          val renumberedQuery = query transformExpressions {
    +            case a@Alias(c, n) if a.exprId == origOutput.exprId => Alias(c, n)(exprId = newExprId)
    +          }
    +
    +          val filteredId = NamedExpression.newExprId
    +          val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +          val withIsFiltered = addIsFiltered(renumberedQuery, filteredId)
    +          val aggValRef = renumberedQuery.output.head
    +
    +          // CASE WHEN isFiltered IS NULL THEN COALESCE(aggVal, resultOnZeroTups)
    +          //      WHEN isFiltered THEN CAST(null AS <type of aggVal>)
    +          //      ELSE aggVal END
    +          val caseExpr = Alias(CaseWhen(
    +            Seq((IsNull(isFilteredRef), Coalesce(Seq(aggValRef,
    +              Literal(resultWithZeroTups.getOrElse(null))))),
    --- End diff --
    
    Fixed in my local copy.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63767826
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala ---
    @@ -293,4 +293,65 @@ class SubquerySuite extends QueryTest with SharedSQLContext {
             """.stripMargin),
           Row(3) :: Nil)
       }
    +
    +  test("COUNT bug in WHERE clause (Filter)") {
    --- End diff --
    
    Will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubque...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on the issue:

    https://github.com/apache/spark/pull/13155
  
    @rxin I'll have an updated set of changes in tonight


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubque...

Posted by sameeragarwal <gi...@git.apache.org>.
Github user sameeragarwal commented on the issue:

    https://github.com/apache/spark/pull/13155
  
    @hvanhovell  maybe this broke the build? https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-branch-2.0-compile-maven-scala-2.10/310/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScalarSubque...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the issue:

    https://github.com/apache/spark/pull/13155
  
    @frreiss  did you have a chance to update this? We want to cut an RC asap.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by frreiss <gi...@git.apache.org>.
Github user frreiss commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r64942870
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    +    var topPart = List[LogicalPlan]()
    +    var bottomPart : LogicalPlan = plan
    +    while (! bottomPart.isInstanceOf[Aggregate]) {
    +      topPart = bottomPart :: topPart
    +      bottomPart = bottomPart.children.head
    +    }
    +    (topPart, bottomPart.asInstanceOf[Aggregate])
    +  }
    +
    +  /**
    +   * Rewrite the nodes above the Aggregate in a subquery so that they generate an
    +   * auxiliary column "isFiltered"
    +   * @param subqueryPlan plan before rewrite
    +   * @param filteredId expression ID for the "isFiltered" column
    +   */
    +  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : ExprId) : LogicalPlan = {
    +    val isFilteredRef = AttributeReference("isFiltered", BooleanType)(exprId = filteredId)
    +    val (topPart, aggNode) = splitSubquery(subqueryPlan)
    +    var rewrittenQuery: LogicalPlan = null
    +    if (topPart.size > 0 && topPart.head.isInstanceOf[Filter]) {
    +      // Correlated subquery has a HAVING clause
    +      // Rewrite the Filter into a Project that returns the value of the filtering predicate
    +      val origFilter = topPart.head.asInstanceOf[Filter]
    +      var topRemainder = topPart.tail
    +      val newProjectList =
    +        origFilter.output :+ Alias(origFilter.condition, "isFiltered")(exprId = filteredId)
    +      val filterAsProject = Project(newProjectList, origFilter.child)
    +
    +      rewrittenQuery = filterAsProject
    +      while (topRemainder.size > 0) {
    +        rewrittenQuery = topRemainder.head match {
    +          case Project(origList, _) => Project(origList :+ isFilteredRef, rewrittenQuery)
    +          case SubqueryAlias(alias, _) => SubqueryAlias(alias, rewrittenQuery)
    +        }
    +        topRemainder = topRemainder.tail
    +      }
    +    } else {
    +      // Correlated subquery without HAVING clause
    +      // Add an additional Project that adds a constant value for "isFiltered"
    +      rewrittenQuery = Project(subqueryPlan.output :+ Alias(Literal(false), "isFiltered")
    +      (exprId = filteredId), subqueryPlan)
    --- End diff --
    
    Good idea. Added a constant in my local copy.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r65097106
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
       }
     
       /**
    +   * Statically evaluate an expression containing zero or more placeholders, given a set
    +   * of bindings for placeholder values.
    +   */
    +  private def evalExpr(expr : Expression, bindings : Map[Long, Option[Any]]) : Option[Any] = {
    +    val rewrittenExpr = expr transform {
    +      case r @ AttributeReference(_, dataType, _, _) =>
    +        bindings(r.exprId.id) match {
    +          case Some(v) => Literal.create(v, dataType)
    +          case None => Literal.default(NullType)
    +        }
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate an expression containing one or more aggregates on an empty input.
    +   */
    +  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
    +    // AggregateExpressions are Unevaluable, so we need to replace all aggregates
    +    // in the expression with the value they would return for zero input tuples.
    +    val rewrittenExpr = expr transform {
    +      case a @ AggregateExpression(aggFunc, _, _, resultId) =>
    +        aggFunc.defaultResult.getOrElse(Literal.default(NullType))
    +    }
    +    Option(rewrittenExpr.eval())
    +  }
    +
    +  /**
    +   * Statically evaluate a scalar subquery on an empty input.
    +   *
    +   * <b>WARNING:</b> This method only covers subqueries that pass the checks under
    +   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in
    +   * CheckAnalysis become less restrictive, this method will need to change.
    +   */
    +  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
    +    // Inputs to this method will start with a chain of zero or more SubqueryAlias
    +    // and Project operators, followed by an optional Filter, followed by an
    +    // Aggregate. Traverse the operators recursively.
    +    def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
    +      lp match {
    +        case SubqueryAlias(_, child) => evalPlan(child)
    +        case Filter(condition, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) bindings
    +          else {
    +            val exprResult = evalExpr(condition, bindings).getOrElse(false)
    +              .asInstanceOf[Boolean]
    +            if (exprResult) bindings else Map()
    +          }
    +
    +        case Project(projectList, child) =>
    +          val bindings = evalPlan(child)
    +          if (bindings.size == 0) {
    +            bindings
    +          } else {
    +            projectList.map(ne => (ne.exprId.id, evalExpr(ne, bindings))).toMap
    +          }
    +
    +        case Aggregate(_, aggExprs, _) =>
    +          // Some of the expressions under the Aggregate node are the join columns
    +          // for joining with the outer query block. Fill those expressions in with
    +          // nulls and statically evaluate the remainder.
    +          aggExprs.map(ne => ne match {
    +            case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
    +            case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId.id, None)
    +            case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
    +          }).toMap
    +
    +        case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
    +      }
    +    }
    +
    +    val resultMap = evalPlan(plan)
    +
    +    // By convention, the scalar subquery result is the leftmost field.
    +    resultMap(plan.output.head.exprId.id)
    +  }
    +
    +  /**
    +   * Split the plan for a scalar subquery into the parts above the Aggregate node
    +   * (first part of returned value) and the parts below the Aggregate node, including
    +   * the Aggregate (second part of returned value)
    +   */
    +  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], Aggregate] = {
    --- End diff --
    
    Use `(Seq[LogicalPlan], Aggregate)` instead of `Tuple2[...]`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/13155


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13155#discussion_r63626483
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala ---
    @@ -293,4 +293,65 @@ class SubquerySuite extends QueryTest with SharedSQLContext {
             """.stripMargin),
           Row(3) :: Nil)
       }
    +
    +  test("COUNT bug in WHERE clause (Filter)") {
    +
    +    // Case 1: Canonical example of the COUNT bug
    +    checkAnswer(
    +      sql("select l.a from l where (select count(*) from r where l.a = r.c) < l.a"),
    +      Row(1) :: Row(1) :: Row(3) :: Row(6) :: Nil)
    +
    +    // Case 2: count(*) = 0; could be rewritten to NOT EXISTS but currently uses
    +    // a rewrite that is vulnerable to the COUNT bug
    +    checkAnswer(
    +      sql("select l.a from l where (select count(*) from r where l.a = r.c) = 0"),
    +      Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil)
    +
    +    // Case 3: COUNT bug without a COUNT aggregate
    +    checkAnswer(
    +      sql("select l.a from l where (select sum(r.d) is null from r where l.a = r.c)"),
    +      Row(1) :: Row(1) ::Row(null) :: Row(null) :: Row(6) :: Nil)
    +
    --- End diff --
    
    (Maybe remove these blank lines, https://github.com/databricks/scala-style-guide#blanklines)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-221340027
  
    **[Test build #3013 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3013/consoleFull)** for PR 13155 at commit [`1b4ba5e`](https://github.com/apache/spark/commit/1b4ba5ed629d9b1e72d919d89b3592f7b29f3f3c).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-15370] [SQL] Update RewriteCorrelatedSc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/13155#issuecomment-221344196
  
    **[Test build #3013 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3013/consoleFull)** for PR 13155 at commit [`1b4ba5e`](https://github.com/apache/spark/commit/1b4ba5ed629d9b1e72d919d89b3592f7b29f3f3c).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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