Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/23248#discussion_r240041688 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala --- @@ -131,8 +131,20 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with PredicateHelper { expressions.flatMap(collectEvaluableUDFs) } - def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case plan: LogicalPlan => extract(plan) + def apply(plan: LogicalPlan): LogicalPlan = plan match { + // SPARK-26293: A subquery will be rewritten into join later, and will go through this rule + // eventually. Here we skip subquery, as Python UDF only needs to be extracted once. + case _: Subquery => plan --- End diff -- I'm not sure if it is totally ok to skip `Subquery` for all optimizer rules. For `ExtractPythonUDFs` I think it is ok because `ExtractPythonUDFs` is performed after the rules in `RewriteSubquery`. So we can skip `ExtractPythonUDFs` here and extract Python UDF after the subqueries are rewritten into join. But for the rules which perform before `RewriteSubquery`, if we skip it on `Subquery`, we have no chance to do the rules after the subqueries are rewritten into join.
--- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org