-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-48466][SQL] Create dedicated node for EmptyRelation in AQE #46830
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 4 commits
7f4b874
192a00a
9e65fa5
76a9c26
d4b21ec
d435954
85328c3
61aba24
d0ca187
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,35 @@ | ||
| /* | ||
| * 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.plans.logical | ||
|
|
||
| import org.apache.spark.sql.catalyst.expressions.Attribute | ||
| import org.apache.spark.sql.catalyst.expressions.SortOrder | ||
|
|
||
| case class EmptyRelation(logical: LogicalPlan) extends LeafNode { | ||
| override val isStreaming: Boolean = logical.isStreaming | ||
|
|
||
| override val outputOrdering: Seq[SortOrder] = logical.outputOrdering | ||
|
|
||
| override def output: Seq[Attribute] = logical.output | ||
|
|
||
| override def computeStats(): Statistics = Statistics(sizeInBytes = 0, rowCount = Some(0)) | ||
|
|
||
| override def maxRows: Option[Long] = Some(0) | ||
|
|
||
| override def maxRowsPerPartition: Option[Long] = Some(0) | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,89 @@ | ||
| /* | ||
| * 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 | ||
|
|
||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions.Attribute | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.vectorized.ColumnarBatch | ||
|
|
||
| /** | ||
| * An intermediate placeholder for EmptyRelation planning, will be replaced with | ||
| * EmptyRelationExec eventually. | ||
| */ | ||
| case class EmptyRelationPlanLater(plan: LogicalPlan) extends PlanLaterBase | ||
|
|
||
| /** | ||
| * A leaf node wrapper for propagated empty relation, which preserved the physical plan. | ||
| */ | ||
| case class EmptyRelationExec(plan: SparkPlan) extends LeafExecNode with InputRDDCodegen { | ||
| private val rdd = sparkContext.emptyRDD[InternalRow] | ||
|
|
||
| override def output: Seq[Attribute] = plan.output | ||
|
|
||
| override protected def doExecute(): RDD[InternalRow] = rdd | ||
|
|
||
| override def executeCollect(): Array[InternalRow] = Array.empty | ||
|
|
||
| override def executeTake(limit: Int): Array[InternalRow] = Array.empty | ||
|
|
||
| override def executeTail(limit: Int): Array[InternalRow] = Array.empty | ||
|
|
||
| protected override def doExecuteColumnar(): RDD[ColumnarBatch] = sparkContext.emptyRDD | ||
|
|
||
| override def inputRDD: RDD[InternalRow] = rdd | ||
|
|
||
| override protected val createUnsafeProjection: Boolean = false | ||
|
|
||
| protected override def stringArgs: Iterator[Any] = Iterator(s"[plan_id=$id]") | ||
|
|
||
| override def generateTreeString( | ||
| depth: Int, | ||
| lastChildren: java.util.ArrayList[Boolean], | ||
| append: String => Unit, | ||
| verbose: Boolean, | ||
| prefix: String = "", | ||
| addSuffix: Boolean = false, | ||
| maxFields: Int, | ||
| printNodeId: Boolean, | ||
| indent: Int = 0): Unit = { | ||
| super.generateTreeString(depth, | ||
| lastChildren, | ||
| append, | ||
| verbose, | ||
| prefix, | ||
| addSuffix, | ||
| maxFields, | ||
| printNodeId, | ||
| indent) | ||
| lastChildren.add(true) | ||
| plan.generateTreeString( | ||
| depth + 1, lastChildren, append, verbose, "", false, maxFields, printNodeId, indent) | ||
| lastChildren.remove(lastChildren.size() - 1) | ||
| } | ||
|
|
||
| override def doCanonicalize(): SparkPlan = { | ||
| this.copy(plan = LocalTableScanExec(plan.output, Nil)) | ||
| } | ||
|
|
||
| override protected[sql] def cleanupResources(): Unit = { | ||
| plan.cleanupResources() | ||
| super.cleanupResources() | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.adaptive | |
|
|
||
| import org.apache.spark.sql.catalyst.optimizer.PropagateEmptyRelationBase | ||
| import org.apache.spark.sql.catalyst.planning.ExtractSingleColumnNullAwareAntiJoin | ||
| import org.apache.spark.sql.catalyst.plans.logical.EmptyRelation | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.catalyst.trees.TreePattern.{LOCAL_RELATION, LOGICAL_QUERY_STAGE, TRUE_OR_FALSE_LITERAL} | ||
| import org.apache.spark.sql.execution.aggregate.BaseAggregateExec | ||
|
|
@@ -34,11 +35,14 @@ import org.apache.spark.sql.execution.joins.HashedRelationWithAllNullKeys | |
| */ | ||
| object AQEPropagateEmptyRelation extends PropagateEmptyRelationBase { | ||
| override protected def isEmpty(plan: LogicalPlan): Boolean = | ||
| super.isEmpty(plan) || (!isRootRepartition(plan) && getEstimatedRowCount(plan).contains(0)) | ||
| super.isEmpty(plan) || plan.isInstanceOf[EmptyRelation] || | ||
|
||
| (!isRootRepartition(plan) && getEstimatedRowCount(plan).contains(0)) | ||
|
|
||
| override protected def nonEmpty(plan: LogicalPlan): Boolean = | ||
| super.nonEmpty(plan) || getEstimatedRowCount(plan).exists(_ > 0) | ||
|
|
||
| override protected def empty(plan: LogicalPlan): LogicalPlan = EmptyRelation(plan) | ||
|
|
||
| private def isRootRepartition(plan: LogicalPlan): Boolean = plan match { | ||
| case l: LogicalQueryStage if l.getTagValue(ROOT_REPARTITION).isDefined => true | ||
| case _ => false | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,36 @@ | ||
| /* | ||
| * 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.adaptive | ||
|
|
||
| import org.apache.spark.sql.Strategy | ||
| import org.apache.spark.sql.catalyst.plans.logical.EmptyRelation | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.execution.EmptyRelationPlanLater | ||
| import org.apache.spark.sql.execution.SparkPlan | ||
|
|
||
| /** | ||
| * A strategy that plan logical [[EmptyRelation]] to physical [[EmptyRelationPlanLater]] which | ||
| * will be planned later. | ||
| */ | ||
| object EmptyRelationPropagationStrategy extends Strategy { | ||
| def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { | ||
| case EmptyRelation(logical) => | ||
| EmptyRelationPlanLater(logical) :: Nil | ||
| case _ => Nil | ||
| } | ||
| } |
Uh oh!
There was an error while loading. Please reload this page.