-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-43124][SQL] Add ConvertCommandResultToLocalRelation rule #45397
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 |
|---|---|---|
| @@ -0,0 +1,52 @@ | ||
| /* | ||
| * 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.catalyst.optimizer | ||
|
|
||
| import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, IntegerLiteral, InterpretedMutableProjection, Predicate, Unevaluable} | ||
| import org.apache.spark.sql.catalyst.plans.logical.{CommandResult, Filter, Limit, LocalRelation, LogicalPlan, Project} | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
| import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND_RESULT | ||
|
|
||
| /** | ||
| * Converts local operations (i.e. ones that don't require data exchange) on `CommandResult` | ||
| * to `LocalRelation`. | ||
| */ | ||
| object ConvertCommandResultToLocalRelation extends Rule[LogicalPlan] { | ||
|
|
||
| override def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning( | ||
| _.containsPattern(COMMAND_RESULT)) { | ||
| case Project(projectList, CommandResult(output, _, _, rows)) | ||
| if !projectList.exists(hasUnevaluableExpr) => | ||
| val projection = new InterpretedMutableProjection(projectList, output) | ||
| projection.initialize(0) | ||
| LocalRelation(projectList.map(_.toAttribute), rows.map(projection(_).copy())) | ||
|
|
||
| case Limit(IntegerLiteral(limit), CommandResult(output, _, _, rows)) => | ||
| LocalRelation(output, rows.take(limit)) | ||
|
|
||
| case Filter(condition, CommandResult(output, _, _, rows)) | ||
|
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. By looking at this rule, I'm on the fence now. The original target of e.g. , if you do I think certain DataFrame operations such as cc @HyukjinKwon
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. ic thanks for explanation.
Member
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. Thanks for the explanation, it makes sense to me, can we continue to review #45373?
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. Yea let's back to #45373 |
||
| if !hasUnevaluableExpr(condition) => | ||
| val predicate = Predicate.create(condition, output) | ||
| predicate.initialize(0) | ||
| LocalRelation(output, rows.filter(row => predicate.eval(row))) | ||
| } | ||
|
|
||
| private def hasUnevaluableExpr(expr: Expression): Boolean = { | ||
| expr.exists(e => e.isInstanceOf[Unevaluable] && !e.isInstanceOf[AttributeReference]) | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,94 @@ | ||
| /* | ||
| * 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.catalyst.optimizer | ||
|
|
||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute | ||
| import org.apache.spark.sql.catalyst.dsl.expressions._ | ||
| import org.apache.spark.sql.catalyst.dsl.plans._ | ||
| import org.apache.spark.sql.catalyst.expressions.{LessThan, Literal} | ||
| import org.apache.spark.sql.catalyst.plans.PlanTest | ||
| import org.apache.spark.sql.catalyst.plans.logical.{CommandResult, LocalRelation, LogicalPlan} | ||
| import org.apache.spark.sql.catalyst.rules.RuleExecutor | ||
|
|
||
| class ConvertCommandResultToLocalRelationSuite extends PlanTest { | ||
|
|
||
| object Optimize extends RuleExecutor[LogicalPlan] { | ||
| val batches = | ||
| Batch("ConvertCommandResultToLocalRelation", FixedPoint(100), | ||
| ConvertCommandResultToLocalRelation, | ||
| ConvertToLocalRelation) :: Nil | ||
| } | ||
|
|
||
| test("Project on CommandResult should be turned into a single LocalRelation") { | ||
| val testCommandResult = CommandResult( | ||
| Seq($"a".int, $"b".int), | ||
| null, | ||
| null, | ||
| InternalRow(1, 2) :: InternalRow(4, 5) :: Nil) | ||
|
|
||
| val correctAnswer = LocalRelation( | ||
| LocalRelation($"a1".int, $"b1".int).output, | ||
| InternalRow(1, 3) :: InternalRow(4, 6) :: Nil) | ||
|
|
||
| val projectOnLocal = testCommandResult.select( | ||
| UnresolvedAttribute("a").as("a1"), | ||
| (UnresolvedAttribute("b") + 1).as("b1")) | ||
|
|
||
| val optimized = Optimize.execute(projectOnLocal.analyze) | ||
|
|
||
| comparePlans(optimized, correctAnswer) | ||
| } | ||
|
|
||
| test("Filter on CommandResult should be turned into a single LocalRelation") { | ||
| val testCommandResult = CommandResult( | ||
| Seq($"a".int, $"b".int), | ||
| null, | ||
| null, | ||
| InternalRow(1, 2) :: InternalRow(4, 5) :: Nil) | ||
|
|
||
| val correctAnswer = LocalRelation( | ||
| LocalRelation($"a1".int, $"b1".int).output, | ||
| InternalRow(1, 3) :: Nil) | ||
|
|
||
| val filterAndProjectOnLocal = testCommandResult | ||
| .select(UnresolvedAttribute("a").as("a1"), (UnresolvedAttribute("b") + 1).as("b1")) | ||
| .where(LessThan(UnresolvedAttribute("b1"), Literal.create(6))) | ||
|
|
||
| val optimized = Optimize.execute(filterAndProjectOnLocal.analyze) | ||
|
|
||
| comparePlans(optimized, correctAnswer) | ||
| } | ||
|
|
||
| test("SPARK-27798: Expression reusing output shouldn't override values in local relation") { | ||
| val testCommandResult = CommandResult( | ||
| Seq($"a".int), | ||
| null, | ||
| null, | ||
| InternalRow(1) :: InternalRow(2) :: Nil) | ||
|
|
||
| val correctAnswer = LocalRelation( | ||
| LocalRelation($"a".struct($"a1".int)).output, | ||
| InternalRow(InternalRow(1)) :: InternalRow(InternalRow(2)) :: Nil) | ||
|
|
||
| val projected = testCommandResult.select(ExprReuseOutput(UnresolvedAttribute("a")).as("a")) | ||
| val optimized = Optimize.execute(projected.analyze) | ||
|
|
||
| comparePlans(optimized, correctAnswer) | ||
| } | ||
| } |
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.
can't we just update
ConvertToLocalRelation?Uh oh!
There was an error while loading. Please reload this page.
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.
Since
CommandResultclass is inspark-sqlmodule, we cannot import it inConvertToLocalRelation(which is inspark-catalyst)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.
And we cannot move
CommandResultintospark-catalystmodule because it usesSparkPlan.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.
oh I see!
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.
Probably, we can add a new trait
LocalRelationConverableto letLocalRelationandCommandResultinherit it.