-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-26293][SQL] Cast exception when having python udf in subquery #23248
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -60,8 +60,12 @@ private class BatchIterator[T](iter: Iterator[T], batchSize: Int) | |
| /** | ||
| * A logical plan that evaluates a [[PythonUDF]]. | ||
| */ | ||
| case class ArrowEvalPython(udfs: Seq[PythonUDF], output: Seq[Attribute], child: LogicalPlan) | ||
| extends UnaryNode | ||
| case class ArrowEvalPython( | ||
| udfs: Seq[PythonUDF], | ||
| output: Seq[Attribute], | ||
| child: LogicalPlan) extends UnaryNode { | ||
| override def producedAttributes: AttributeSet = AttributeSet(output.drop(child.output.length)) | ||
|
||
| } | ||
|
|
||
| /** | ||
| * A physical plan that evaluates a [[PythonUDF]]. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,7 +24,7 @@ import org.apache.spark.api.python.PythonEvalType | |
| import org.apache.spark.sql.AnalysisException | ||
| import org.apache.spark.sql.catalyst.expressions._ | ||
| import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression | ||
| import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LogicalPlan, Project} | ||
| import org.apache.spark.sql.catalyst.plans.logical._ | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
|
|
||
|
|
||
|
|
@@ -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 | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Personally I found it a bit confusing when two seeming unrelated things are put together (Subquery and ExtractPythonUDFs). I wonder if it's sufficient to make ExtractPythonUDFs idempotent?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I agree it's a bit confusing, but that's how It's sufficient to make In general, I think we should skip
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see. If it's common to skip Subquery in other rules, I guess it's ok to put it in here as well. But it would definitely be helpful to establish some kind of guidance, maybe sth like "All optimizer rule should skip Subquery because OptimizeSubqueries will execute them anyway"?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think you have a point here. If subquery will be converted to join, why do we need to optimize subquery ahead? Anyway, that's something we need to discuss later. cc @dilipbiswal for the subquery question.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm not sure if it is totally ok to skip For But for the rules which perform before
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Basically, we want to ensure this rule is running once and only once. In the future, if we have another rule/function that calls Optimizer.this.execute(plan), this rule needs to be fixed again... We have a very strong hidden assumption in the implementation. This looks risky in the long term. The current fix is fine for backporting to 2.4. |
||
|
|
||
| case _ => plan transformUp { | ||
| // A safe guard. `ExtractPythonUDFs` only runs once, so we will not hit `BatchEvalPython` and | ||
| // `ArrowEvalPython` in the input plan. However if we hit them, we must skip them, as we can't | ||
| // extract Python UDFs from them. | ||
| case p: BatchEvalPython => p | ||
| case p: ArrowEvalPython => p | ||
|
|
||
| case plan: LogicalPlan => extract(plan) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
add the import here, as a lof of tests use it
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah, yea. It's okay and I think it's good timing to clean up while we are here, and while it's broken down into multiple test files now.