Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -67,4 +67,10 @@ case class PythonUDF(
exprId = resultId)

override def nullable: Boolean = true

override lazy val canonicalized: Expression = {
val canonicalizedChildren = children.map(_.canonicalized)
// `resultId` can be seen as cosmetic variation in PythonUDF, as it doesn't affect the result.
Canonicalize.execute(this.copy(resultId = ExprId(-1)).withNewChildren(canonicalizedChildren))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we still need to run Canonicalize.execute?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this can be saved, yes.

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.catalog.SessionCatalog
import org.apache.spark.sql.catalyst.optimizer.{ColumnPruning, Optimizer, PushPredicateThroughNonJoin, RemoveNoopOperators}
import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions
import org.apache.spark.sql.execution.datasources.SchemaPruning
import org.apache.spark.sql.execution.python.{ExtractPythonUDFFromAggregate, ExtractPythonUDFs}
import org.apache.spark.sql.execution.python.{ExtractGroupingPythonUDFFromAggregate, ExtractPythonUDFFromAggregate, ExtractPythonUDFs}

class SparkOptimizer(
catalog: SessionCatalog,
Expand All @@ -33,6 +33,7 @@ class SparkOptimizer(
Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+
Batch("Extract Python UDFs", Once,
ExtractPythonUDFFromAggregate,
ExtractGroupingPythonUDFFromAggregate,
ExtractPythonUDFs,
// The eval-python node may be between Project/Filter and the scan node, which breaks
// column pruning and filter push-down. Here we rerun the related optimizer rules.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,64 @@ object ExtractPythonUDFFromAggregate extends Rule[LogicalPlan] {
}
}

/**
* Extracts PythonUDFs in logical aggregate, which are used in grouping keys, evaluate them
* before aggregate.
* This must be executed after `ExtractPythonUDFFromAggregate` rule and before `ExtractPythonUDFs`.
*/
object ExtractGroupingPythonUDFFromAggregate extends Rule[LogicalPlan] {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what's the difference between this rule and ExtractPythonUDFFromAggregate?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ExtractPythonUDFFromAggregate pulls out Python UDFs which have aggregate expression or grouping key as input, like udf(sum(c)), and Python UDFs which have no input. Those UDFs pulled out are evaluated after aggregate.

This rule, ExtractGroupingPythonUDFFromAggregate, pulls out Python UDFs which are used in grouping keys, like SELECT count(*) FROM table GROUP BY udf(id). This kind of Python UDF is evaluated before aggregate.

private def hasScalarPythonUDF(e: Expression): Boolean = {
e.find(PythonUDF.isScalarPythonUDF).isDefined
}

private def extract(agg: Aggregate): LogicalPlan = {
val projList = new ArrayBuffer[NamedExpression]()
val groupingExpr = new ArrayBuffer[Expression]()
val attributeMap = mutable.HashMap[PythonUDF, NamedExpression]()

agg.groupingExpressions.foreach { expr =>
if (hasScalarPythonUDF(expr)) {
val newE = expr transformDown {
case p: PythonUDF =>
// This is just a sanity check, the rule PullOutNondeterministic should
// already pull out those nondeterministic expressions.
assert(p.udfDeterministic, "Non-determinstic PythonUDFs should not appear " +
"in grouping expression")
val alias = Alias(p, "groupingPythonUDF")()
projList += alias
attributeMap += ((p.canonicalized.asInstanceOf[PythonUDF], alias))
alias.toAttribute
}
groupingExpr += newE
} else {
groupingExpr += expr
}
}
val aggExpr = agg.aggregateExpressions.map { expr =>
expr.transformUp {
// PythonUDF over aggregate was pull out by ExtractPythonUDFFromAggregate.
// PythonUDF here should be either
// 1. Argument of an aggregate function.
// CheckAnalysis guarantees the arguments are deterministic.
// 2. PythonUDF in grouping key. Grouping key must be deterministic.
// 3. PythonUDF not in grouping key. It is either no arguments or with grouping key
// in its arguments. Such PythonUDF was pull out by ExtractPythonUDFFromAggregate, too.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

tiny nit: spacing.

case p: PythonUDF if p.udfDeterministic =>
val canonicalized = p.canonicalized.asInstanceOf[PythonUDF]
attributeMap.get(canonicalized).map(_.toAttribute).getOrElse(p)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can put attributes in attributeMap, instead of Alias.

}.asInstanceOf[NamedExpression]
}
agg.copy(
groupingExpressions = groupingExpr,
aggregateExpressions = aggExpr,
child = Project(projList ++ agg.child.output, agg.child))
}

def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
case agg: Aggregate if agg.groupingExpressions.exists(hasScalarPythonUDF(_)) =>
extract(agg)
}
}

/**
* Extracts PythonUDFs from operators, rewriting the query plan so that the UDF can be evaluated
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution.python

import org.apache.spark.sql.{IntegratedUDFTestUtils, QueryTest}
import org.apache.spark.sql.functions.count
import org.apache.spark.sql.test.SharedSQLContext

class PythonUDFSuite extends QueryTest with SharedSQLContext {
import testImplicits._

test("SPARK-28445: PythonUDF in grouping key and aggregate expressions") {
import IntegratedUDFTestUtils._

val scalaTestUDF = TestScalaUDF(name = "scalaUDF")
val pythonTestUDF = TestPythonUDF(name = "pyUDF")
assume(shouldTestPythonUDFs)

val base = Seq(
(Some(1), Some(1)), (Some(1), Some(2)), (Some(2), Some(1)),
(Some(2), Some(2)), (Some(3), Some(1)), (Some(3), Some(2)),
(None, Some(1)), (Some(3), None), (None, None)).toDF("a", "b")

val df = base.groupBy(scalaTestUDF(base("a") + 1))
.agg(scalaTestUDF(base("a") + 1), scalaTestUDF(count(base("b"))))
val df2 = base.groupBy(pythonTestUDF(base("a") + 1))
.agg(pythonTestUDF(base("a") + 1), pythonTestUDF(count(base("b"))))
checkAnswer(df, df2)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we create a test case for each of these checks? We can move the scalaTestUDF, pythonTestUDF and base to the class body.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok.


val df3 = base.groupBy(scalaTestUDF(base("a") + 1))
.agg(scalaTestUDF(base("a") + 1) + 1, scalaTestUDF(count(base("b"))))
val df4 = base.groupBy(pythonTestUDF(base("a") + 1))
.agg(pythonTestUDF(base("a") + 1) + 1, pythonTestUDF(count(base("b"))))
checkAnswer(df3, df4)

// PythonUDF in aggregate expression has grouping key in its arguments.
val df5 = base.groupBy(scalaTestUDF(base("a") + 1))
.agg(scalaTestUDF(scalaTestUDF(base("a") + 1)), scalaTestUDF(count(base("b"))))
val df6 = base.groupBy(pythonTestUDF(base("a") + 1))
.agg(pythonTestUDF(pythonTestUDF(base("a") + 1)), pythonTestUDF(count(base("b"))))
checkAnswer(df5, df6)

// PythonUDF over grouping key is argument to aggregate function.
val df7 = base.groupBy(scalaTestUDF(base("a") + 1))
.agg(scalaTestUDF(scalaTestUDF(base("a") + 1)),
scalaTestUDF(count(scalaTestUDF(base("a") + 1))))
val df8 = base.groupBy(pythonTestUDF(base("a") + 1))
.agg(pythonTestUDF(pythonTestUDF(base("a") + 1)),
pythonTestUDF(count(pythonTestUDF(base("a") + 1))))
checkAnswer(df7, df8)
}
}