From 6bfdb327e643e644b72b77ba1898ae9f3600101e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 1 Aug 2016 11:41:34 +0800 Subject: [PATCH 01/28] Dedup common subqueries. --- .../sql/catalyst/optimizer/Optimizer.scala | 40 +++++++++++- .../spark/sql/execution/QueryExecution.scala | 3 +- .../spark/sql/execution/SparkStrategies.scala | 2 + .../apache/spark/sql/execution/subquery.scala | 16 ++++- .../subquery/CommonSubqueryAlias.scala | 55 ++++++++++++++++ .../subquery/CommonSubqueryExec.scala | 51 +++++++++++++++ .../execution/subquery/SubqueryDedup.scala | 64 +++++++++++++++++++ .../org/apache/spark/sql/SubquerySuite.scala | 20 ++++++ .../subquery/SubqueryDedupSuite.scala | 51 +++++++++++++++ 9 files changed, 299 insertions(+), 3 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryAlias.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/subquery/SubqueryDedupSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index fe328fd598d71..66bbecaeb6ec8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -49,7 +49,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) // However, because we also use the analyzer to canonicalized queries (for view definition), // we do not eliminate subqueries or compute current time in the analyzer. Batch("Finish Analysis", Once, - EliminateSubqueryAliases, + EliminateOneTimeSubqueryAliases, ReplaceExpressions, ComputeCurrentTime, GetCurrentDatabase(sessionCatalog), @@ -2052,3 +2052,41 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { } } } + +/** + * Removes the [[SubqueryAlias]] operators which are used only once from the plan and use + * [[CommonSubqueryAlias]] to replace these operators. + */ +object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + val subqueries = ArrayBuffer[LogicalPlan]() + val duplicateSubqueries = ArrayBuffer[LogicalPlan]() + + val noRecursiveSubqueryPlan = plan.transformDown { + // Eliminate the recursive subqueries which have the same output. + case s @ SubqueryAlias(_, child) + if child.find(p => p.isInstanceOf[SubqueryAlias] && p.sameResult(s)).isDefined => + child + } + + noRecursiveSubqueryPlan.foreach { + // Collect the subqueries that are used more than once in the query. + case SubqueryAlias(_, child) => + if (subqueries.indexWhere(s => s.sameResult(child)) >= 0) { + // If the plan with same results can be found. + duplicateSubqueries += child + } else { + // If it can't be found, add it into the list. + subqueries += child + } + case _ => + } + + noRecursiveSubqueryPlan.transformDown { + // Only eliminate the subqueries that are used only once in the query. + case SubqueryAlias(_, child) + if duplicateSubqueries.indexWhere(s => s.sameResult(child)) < 0 => + child + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 5b9af26dfc4f8..2c6a5efdb9408 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -77,7 +77,8 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { SparkSession.setActiveSession(sparkSession) // TODO: We use next(), i.e. take the first plan returned by the planner, here for now, // but we will implement to choose the best plan. - planner.plan(ReturnAnswer(optimizedPlan)).next() + val dedupSubqueries = DedupCommonSubqueries(sparkSession)(optimizedPlan) + planner.plan(ReturnAnswer(dedupSubqueries)).next() } // executedPlan should not be used to initialize any SparkPlan. It should be diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 52e19819f2f61..390b23ca1034a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight} import org.apache.spark.sql.execution.streaming.{MemoryPlan, StreamingExecutionRelation, StreamingRelation, StreamingRelationExec} +import org.apache.spark.sql.execution.subquery.{CommonSubqueryAlias, CommonSubqueryExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQuery @@ -414,6 +415,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil case LogicalRDD(output, rdd) => RDDScanExec(output, rdd, "ExistingRDD") :: Nil case BroadcastHint(child) => planLater(child) :: Nil + case c: CommonSubqueryAlias => CommonSubqueryExec(c.output, c) :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 461d3010ada7e..6aa58190e5a1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -22,8 +22,9 @@ import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, ExprId, Literal, SubqueryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.subquery.SubqueryDedup import org.apache.spark.sql.types.DataType /** @@ -79,3 +80,16 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { } } } + +/** + * Dedup common subqueries which are used more than once in the given [[LogicalPlan]]. + */ +case class DedupCommonSubqueries(sparkSession: SparkSession) extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + val dedup = new SubqueryDedup() + plan.transformDown { + case SubqueryAlias(_, child) => + dedup.createCommonSubquery(sparkSession, child) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryAlias.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryAlias.scala new file mode 100644 index 0000000000000..b7d586b5866b5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryAlias.scala @@ -0,0 +1,55 @@ +/* + * 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.subquery + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical.Statistics +import org.apache.spark.sql.execution.SparkPlan + +private[sql] case class CommonSubqueryAlias( + output: Seq[Attribute], + @transient child: SparkPlan)( + private[sql] val _statistics: Statistics, + @transient private[sql] var _computedOutput: RDD[InternalRow] = null) + extends logical.LeafNode { + + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) + + override def producedAttributes: AttributeSet = outputSet + + override lazy val statistics: Statistics = _statistics + + if (_computedOutput == null) { + _computedOutput = child.execute() + } + + lazy val numRows: Long = computedOutput.count + + def withOutput(newOutput: Seq[Attribute]): CommonSubqueryAlias = { + CommonSubqueryAlias(newOutput, child)(_statistics, _computedOutput) + } + + def computedOutput: RDD[InternalRow] = _computedOutput + + override protected def otherCopyArgs: Seq[AnyRef] = + Seq(_statistics, _computedOutput) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala new file mode 100644 index 0000000000000..a377fea0d2bbb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala @@ -0,0 +1,51 @@ +/* + * 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.subquery + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.execution.metric.SQLMetrics + + +private[sql] case class CommonSubqueryExec( + attributes: Seq[Attribute], + @transient subquery: CommonSubqueryAlias) + extends LeafExecNode { + + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(subquery) ++ super.innerChildren + + private[sql] override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + + override def output: Seq[Attribute] = attributes + + override def outputPartitioning: Partitioning = subquery.child.outputPartitioning + + override def outputOrdering: Seq[SortOrder] = subquery.child.outputOrdering + + protected override def doExecute(): RDD[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + val subqueryOutput = subquery.computedOutput + numOutputRows += subquery.numRows + subqueryOutput + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala new file mode 100644 index 0000000000000..86471d5bef21c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala @@ -0,0 +1,64 @@ +/* + * 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.subquery + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.SparkSession + +/** Holds a subquery logical plan and its data */ +private[sql] case class CommonSubquery(plan: LogicalPlan, subquery: CommonSubqueryAlias) + +/** + * Provides support to eliminate unnecessary subqueries execution. Unlike the queries that need to + * be cached with [[CacheManager]], the subqueries are only used in a single query but could be + * executed multiple times during the query execution. [[CommonSubqueryAlias]] is used to keep the + * common [[SparkPlan]] for the duplicate subqueries in a query. + */ +private[sql] class SubqueryDedup { + + private val subqueryData = new scala.collection.mutable.ArrayBuffer[CommonSubquery] + + /** + * Creates a [[CommonSubqueryAlias]] for the given logical plan. A [[CommonSubqueryAlias]] + * wraps the output of the logical plan, the [[SparkPlan]] of the logical plan and its statistic. + * This method will first look up if there is already logical plan with the same results in the + * subqueries list. If so, it returns the previously created [[CommonSubqueryAlias]]. If not, + * this method will create a new [[CommonSubqueryAlias]]. Thus, all the logical plans which + * produce the same results, will refer to the same [[SparkPlan]] which will be executed + * only once at running stage. + */ + private[sql] def createCommonSubquery( + sparkSession: SparkSession, + planToDedup: LogicalPlan): CommonSubqueryAlias = { + lookupCommonSubquery(planToDedup).map(_.subquery).getOrElse { + val common = + CommonSubquery( + planToDedup, + CommonSubqueryAlias(planToDedup.output, + sparkSession.sessionState.executePlan(planToDedup).executedPlan) + (planToDedup.statistics)) + subqueryData += common + common.subquery + }.withOutput(planToDedup.output) + } + + /** Optionally returns common subquery for the given [[LogicalPlan]]. */ + private[sql] def lookupCommonSubquery(plan: LogicalPlan): Option[CommonSubquery] = { + subqueryData.find(cd => plan.sameResult(cd.plan)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index afed342ff8e2a..0594e16643ab0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import org.apache.spark.sql.execution.subquery.CommonSubqueryExec +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext class SubquerySuite extends QueryTest with SharedSQLContext { @@ -571,4 +573,22 @@ class SubquerySuite extends QueryTest with SharedSQLContext { Row(1.0, false) :: Row(1.0, false) :: Row(2.0, true) :: Row(2.0, true) :: Row(3.0, false) :: Row(5.0, true) :: Row(null, false) :: Row(null, true) :: Nil) } + + test("Dedup subqueries") { + spark.range(10).createOrReplaceTempView("t") + val df = sql("WITH s AS (SELECT 1 FROM t) SELECT * FROM s s1 join s s2") + + val commonSubqueries = df.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries.length == 1) + + val df2 = sql("WITH s1 AS (SELECT 1 FROM t), s2 AS (SELECT 1 FROM t) " + + "SELECT * FROM s1 JOIN (SELECT * FROM s1, s2)") + + val commonSubqueries2 = df2.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries2.length == 1) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/subquery/SubqueryDedupSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/subquery/SubqueryDedupSuite.scala new file mode 100644 index 0000000000000..1477167687520 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/subquery/SubqueryDedupSuite.scala @@ -0,0 +1,51 @@ +/* + * 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.subquery + +import org.apache.spark.sql.{Dataset, QueryTest} +import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.test.SharedSQLContext + +class SubqueryDedupSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("Create common subquery") { + val subqueryDedup = new SubqueryDedup() + + spark.range(10).createOrReplaceTempView("t1") + val df = sql("SELECT 1 FROM t1") + val project = df.queryExecution.analyzed.collect { + case p: Project => p + }.head + + val commonSubqueryAlias = subqueryDedup.createCommonSubquery(spark, project) + assert(commonSubqueryAlias.output === project.output) + assert(Dataset.ofRows(spark, commonSubqueryAlias).collect() === df.collect()) + + spark.range(10).createOrReplaceTempView("t2") + + val df2 = sql("SELECT * FROM t1 join t2") + val project2 = df2.queryExecution.analyzed.collect { + case p: Project => p + }.head + + val commonSubqueryAlias2 = subqueryDedup.createCommonSubquery(spark, project2) + assert(commonSubqueryAlias2.output === project2.output) + assert(Dataset.ofRows(spark, commonSubqueryAlias2).collect() === df2.collect()) + } +} From 229ae31fb724d15ef8e1911c79da63121fc1701d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 4 Aug 2016 17:25:04 +0800 Subject: [PATCH 02/28] Add logical node CommonSubqueryAlias to represent subquery in CTE. --- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 11 +++++----- .../sql/catalyst/parser/AstBuilder.scala | 4 ++-- .../plans/logical/basicLogicalOperators.scala | 8 +++++++- .../sql/catalyst/parser/PlanParserSuite.scala | 2 +- .../spark/sql/catalyst/SQLBuilder.scala | 2 ++ .../spark/sql/execution/QueryExecution.scala | 10 +++++++--- .../spark/sql/execution/SparkStrategies.scala | 4 ++-- .../apache/spark/sql/execution/subquery.scala | 6 +++--- ...bqueryAlias.scala => CommonSubquery.scala} | 6 +++--- .../subquery/CommonSubqueryExec.scala | 2 +- .../execution/subquery/SubqueryDedup.scala | 20 +++++++++---------- .../org/apache/spark/sql/SubquerySuite.scala | 8 ++++++++ 13 files changed, 52 insertions(+), 33 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/{CommonSubqueryAlias.scala => CommonSubquery.scala} (91%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 61162ccdba810..8ee5c3ab84203 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -138,7 +138,7 @@ class Analyzer( // see https://github.com/apache/spark/pull/4929#discussion_r27186638 for more info case u : UnresolvedRelation => val substituted = cteRelations.get(u.tableIdentifier.table).map { relation => - val withAlias = u.alias.map(SubqueryAlias(_, relation)) + val withAlias = u.alias.map(CommonSubqueryAlias(_, relation)) withAlias.getOrElse(relation) } substituted.getOrElse(u) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 66bbecaeb6ec8..818d7c3d6ffe8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -49,7 +49,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) // However, because we also use the analyzer to canonicalized queries (for view definition), // we do not eliminate subqueries or compute current time in the analyzer. Batch("Finish Analysis", Once, - EliminateOneTimeSubqueryAliases, + EliminateSubqueryAliases, ReplaceExpressions, ComputeCurrentTime, GetCurrentDatabase(sessionCatalog), @@ -2054,8 +2054,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { } /** - * Removes the [[SubqueryAlias]] operators which are used only once from the plan and use - * [[CommonSubqueryAlias]] to replace these operators. + * Removes the [[CommonSubqueryAlias]] operators which are used only once from the plan. */ object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { @@ -2064,14 +2063,14 @@ object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { val noRecursiveSubqueryPlan = plan.transformDown { // Eliminate the recursive subqueries which have the same output. - case s @ SubqueryAlias(_, child) + case s @ CommonSubqueryAlias(_, child) if child.find(p => p.isInstanceOf[SubqueryAlias] && p.sameResult(s)).isDefined => child } noRecursiveSubqueryPlan.foreach { // Collect the subqueries that are used more than once in the query. - case SubqueryAlias(_, child) => + case CommonSubqueryAlias(_, child) => if (subqueries.indexWhere(s => s.sameResult(child)) >= 0) { // If the plan with same results can be found. duplicateSubqueries += child @@ -2084,7 +2083,7 @@ object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { noRecursiveSubqueryPlan.transformDown { // Only eliminate the subqueries that are used only once in the query. - case SubqueryAlias(_, child) + case CommonSubqueryAlias(_, child) if duplicateSubqueries.indexWhere(s => s.sameResult(child)) < 0 => child } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index f2cc8d362478a..7b8b1ed88fc30 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -106,8 +106,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * * This is only used for Common Table Expressions. */ - override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) { - SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith)) + override def visitNamedQuery(ctx: NamedQueryContext): CommonSubqueryAlias = withOrigin(ctx) { + CommonSubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith)) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index b31f5aa11c229..e7e6038c54abe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -396,7 +396,8 @@ case class InsertIntoTable( * key is the alias of the CTE definition, * value is the CTE definition. */ -case class With(child: LogicalPlan, cteRelations: Map[String, SubqueryAlias]) extends UnaryNode { +case class With(child: LogicalPlan, cteRelations: Map[String, CommonSubqueryAlias]) + extends UnaryNode { override def output: Seq[Attribute] = child.output } @@ -694,6 +695,11 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNo } } +case class CommonSubqueryAlias(alias: String, child: LogicalPlan) extends UnaryNode { + + override def output: Seq[Attribute] = child.output.map(_.withQualifier(Some(alias))) +} + case class SubqueryAlias(alias: String, child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output.map(_.withQualifier(Some(alias))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index fbe236e196268..e8298a6a6dc23 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -77,7 +77,7 @@ class PlanParserSuite extends PlanTest { def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = { val ctes = namedPlans.map { case (name, cte) => - name -> SubqueryAlias(name, cte) + name -> CommonSubqueryAlias(name, cte) }.toMap With(plan, ctes) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index 9a02e3c8f31f9..d67bb066c1f4a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -154,6 +154,8 @@ class SQLBuilder(logicalPlan: LogicalPlan) extends Logging { case p: SubqueryAlias => build("(" + toSQL(p.child) + ")", "AS", p.alias) + case p: CommonSubqueryAlias => build("(" + toSQL(p.child) + ")", "AS", p.alias) + case p: Join => build( toSQL(p.left), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 2c6a5efdb9408..e25ee384055d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -24,6 +24,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker +import org.apache.spark.sql.catalyst.optimizer.EliminateOneTimeSubqueryAliases import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -71,14 +72,17 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { sparkSession.sharedState.cacheManager.useCachedData(analyzed) } - lazy val optimizedPlan: LogicalPlan = sparkSession.sessionState.optimizer.execute(withCachedData) + lazy val optimizedPlan: LogicalPlan = { + val noOneTimeCommonSubqueries = EliminateOneTimeSubqueryAliases(withCachedData) + val dedupSubqueries = DedupCommonSubqueries(sparkSession)(noOneTimeCommonSubqueries) + sparkSession.sessionState.optimizer.execute(dedupSubqueries) + } lazy val sparkPlan: SparkPlan = { SparkSession.setActiveSession(sparkSession) // TODO: We use next(), i.e. take the first plan returned by the planner, here for now, // but we will implement to choose the best plan. - val dedupSubqueries = DedupCommonSubqueries(sparkSession)(optimizedPlan) - planner.plan(ReturnAnswer(dedupSubqueries)).next() + planner.plan(ReturnAnswer(optimizedPlan)).next() } // executedPlan should not be used to initialize any SparkPlan. It should be diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 390b23ca1034a..1f849b753990f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight} import org.apache.spark.sql.execution.streaming.{MemoryPlan, StreamingExecutionRelation, StreamingRelation, StreamingRelationExec} -import org.apache.spark.sql.execution.subquery.{CommonSubqueryAlias, CommonSubqueryExec} +import org.apache.spark.sql.execution.subquery.{CommonSubquery, CommonSubqueryExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQuery @@ -415,7 +415,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil case LogicalRDD(output, rdd) => RDDScanExec(output, rdd, "ExistingRDD") :: Nil case BroadcastHint(child) => planLater(child) :: Nil - case c: CommonSubqueryAlias => CommonSubqueryExec(c.output, c) :: Nil + case c: CommonSubquery => CommonSubqueryExec(c.output, c) :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 6aa58190e5a1d..3841fd93623f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, ExprId, Literal, SubqueryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} +import org.apache.spark.sql.catalyst.plans.logical.{CommonSubqueryAlias, Filter, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.subquery.SubqueryDedup import org.apache.spark.sql.types.DataType @@ -88,8 +88,8 @@ case class DedupCommonSubqueries(sparkSession: SparkSession) extends Rule[Logica def apply(plan: LogicalPlan): LogicalPlan = { val dedup = new SubqueryDedup() plan.transformDown { - case SubqueryAlias(_, child) => - dedup.createCommonSubquery(sparkSession, child) + case s: CommonSubqueryAlias => + dedup.createCommonSubquery(sparkSession, s.child) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryAlias.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala similarity index 91% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryAlias.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala index b7d586b5866b5..e691429c458b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryAlias.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.SparkPlan -private[sql] case class CommonSubqueryAlias( +private[sql] case class CommonSubquery( output: Seq[Attribute], @transient child: SparkPlan)( private[sql] val _statistics: Statistics, @@ -44,8 +44,8 @@ private[sql] case class CommonSubqueryAlias( lazy val numRows: Long = computedOutput.count - def withOutput(newOutput: Seq[Attribute]): CommonSubqueryAlias = { - CommonSubqueryAlias(newOutput, child)(_statistics, _computedOutput) + def withOutput(newOutput: Seq[Attribute]): CommonSubquery = { + CommonSubquery(newOutput, child)(_statistics, _computedOutput) } def computedOutput: RDD[InternalRow] = _computedOutput diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala index a377fea0d2bbb..7afdd527b8461 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.metric.SQLMetrics private[sql] case class CommonSubqueryExec( attributes: Seq[Attribute], - @transient subquery: CommonSubqueryAlias) + @transient subquery: CommonSubquery) extends LeafExecNode { override protected def innerChildren: Seq[QueryPlan[_]] = Seq(subquery) ++ super.innerChildren diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala index 86471d5bef21c..1625378dcee43 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala @@ -21,35 +21,35 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.SparkSession /** Holds a subquery logical plan and its data */ -private[sql] case class CommonSubquery(plan: LogicalPlan, subquery: CommonSubqueryAlias) +private[sql] case class CommonSubqueryItem(plan: LogicalPlan, subquery: CommonSubquery) /** * Provides support to eliminate unnecessary subqueries execution. Unlike the queries that need to * be cached with [[CacheManager]], the subqueries are only used in a single query but could be - * executed multiple times during the query execution. [[CommonSubqueryAlias]] is used to keep the + * executed multiple times during the query execution. [[CommonSubquery]] is used to keep the * common [[SparkPlan]] for the duplicate subqueries in a query. */ private[sql] class SubqueryDedup { - private val subqueryData = new scala.collection.mutable.ArrayBuffer[CommonSubquery] + private val subqueryData = new scala.collection.mutable.ArrayBuffer[CommonSubqueryItem] /** - * Creates a [[CommonSubqueryAlias]] for the given logical plan. A [[CommonSubqueryAlias]] + * Creates a [[CommonSubquery]] for the given logical plan. A [[CommonSubquery]] * wraps the output of the logical plan, the [[SparkPlan]] of the logical plan and its statistic. * This method will first look up if there is already logical plan with the same results in the - * subqueries list. If so, it returns the previously created [[CommonSubqueryAlias]]. If not, - * this method will create a new [[CommonSubqueryAlias]]. Thus, all the logical plans which + * subqueries list. If so, it returns the previously created [[CommonSubquery]]. If not, + * this method will create a new [[CommonSubquery]]. Thus, all the logical plans which * produce the same results, will refer to the same [[SparkPlan]] which will be executed * only once at running stage. */ private[sql] def createCommonSubquery( sparkSession: SparkSession, - planToDedup: LogicalPlan): CommonSubqueryAlias = { + planToDedup: LogicalPlan): CommonSubquery = { lookupCommonSubquery(planToDedup).map(_.subquery).getOrElse { val common = - CommonSubquery( + CommonSubqueryItem( planToDedup, - CommonSubqueryAlias(planToDedup.output, + CommonSubquery(planToDedup.output, sparkSession.sessionState.executePlan(planToDedup).executedPlan) (planToDedup.statistics)) subqueryData += common @@ -58,7 +58,7 @@ private[sql] class SubqueryDedup { } /** Optionally returns common subquery for the given [[LogicalPlan]]. */ - private[sql] def lookupCommonSubquery(plan: LogicalPlan): Option[CommonSubquery] = { + private[sql] def lookupCommonSubquery(plan: LogicalPlan): Option[CommonSubqueryItem] = { subqueryData.find(cd => plan.sameResult(cd.plan)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 0594e16643ab0..70d6f85ec12e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -590,5 +590,13 @@ class SubquerySuite extends QueryTest with SharedSQLContext { case c: CommonSubqueryExec => c.subquery.child }.distinct assert(commonSubqueries2.length == 1) + + val df3 = + sql("WITH t1 AS (SELECT 1 AS id FROM t) SELECT * FROM t1 a, t1 b WHERE a.id = 1 AND b.id > 0") + + val commonSubqueries3 = df3.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries3.length == 1) } } From 70af8d622c3da2094b861daf5b8adb20bcc489aa Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 10 Aug 2016 11:15:45 +0800 Subject: [PATCH 03/28] Update with the change of access privileges for metrics. --- .../spark/sql/execution/subquery/CommonSubqueryExec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala index 7afdd527b8461..3c45ee0b62dd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala @@ -33,7 +33,7 @@ private[sql] case class CommonSubqueryExec( override protected def innerChildren: Seq[QueryPlan[_]] = Seq(subquery) ++ super.innerChildren - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def output: Seq[Attribute] = attributes From ba11d34335739819e979a934c735afa1cb65b5e8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 10 Aug 2016 11:29:38 +0800 Subject: [PATCH 04/28] Use optimized plan of common subquery. --- .../main/scala/org/apache/spark/sql/execution/subquery.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 3841fd93623f5..11706dc3b5b52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -89,7 +89,8 @@ case class DedupCommonSubqueries(sparkSession: SparkSession) extends Rule[Logica val dedup = new SubqueryDedup() plan.transformDown { case s: CommonSubqueryAlias => - dedup.createCommonSubquery(sparkSession, s.child) + val optimized = sparkSession.sessionState.optimizer.execute(s.child) + dedup.createCommonSubquery(sparkSession, optimized) } } } From 4680994b79959ae9f98bf37b99d0ca3de88a4d5a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 10 Aug 2016 15:21:23 +0800 Subject: [PATCH 05/28] Add test case for self-join. --- .../test/scala/org/apache/spark/sql/SubquerySuite.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 69327376f286e..a179c750c0077 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -598,6 +598,13 @@ class SubquerySuite extends QueryTest with SharedSQLContext { case c: CommonSubqueryExec => c.subquery.child }.distinct assert(commonSubqueries3.length == 1) + + // Using a self-join as CTE to test if de-duplicated attributes work for this. + val df4 = sql("WITH j AS (SELECT * FROM (SELECT * FROM l JOIN l)) SELECT * FROM j j1, j j2") + val commonSubqueries4 = df4.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries4.length == 1) } test("SPARK-16804: Correlated subqueries containing LIMIT - 1") { From 11815daa7d4073b8a3ce845e7c63d84defff017a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 10 Aug 2016 16:15:21 +0800 Subject: [PATCH 06/28] Synchonized on computed output of common subquery. --- .../execution/subquery/CommonSubquery.scala | 11 ++-- .../org/apache/spark/sql/SubquerySuite.scala | 52 +++++++++++-------- 2 files changed, 35 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala index e691429c458b0..a68210afbf529 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala @@ -38,17 +38,18 @@ private[sql] case class CommonSubquery( override lazy val statistics: Statistics = _statistics - if (_computedOutput == null) { - _computedOutput = child.execute() - } - lazy val numRows: Long = computedOutput.count def withOutput(newOutput: Seq[Attribute]): CommonSubquery = { CommonSubquery(newOutput, child)(_statistics, _computedOutput) } - def computedOutput: RDD[InternalRow] = _computedOutput + def computedOutput: RDD[InternalRow] = this.synchronized { + if (_computedOutput == null) { + _computedOutput = child.execute() + } + _computedOutput + } override protected def otherCopyArgs: Seq[AnyRef] = Seq(_statistics, _computedOutput) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index a179c750c0077..9602b206930c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -575,29 +575,31 @@ class SubquerySuite extends QueryTest with SharedSQLContext { } test("Dedup subqueries") { - spark.range(10).createOrReplaceTempView("t") - val df = sql("WITH s AS (SELECT 1 FROM t) SELECT * FROM s s1 join s s2") - - val commonSubqueries = df.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child - }.distinct - assert(commonSubqueries.length == 1) - - val df2 = sql("WITH s1 AS (SELECT 1 FROM t), s2 AS (SELECT 1 FROM t) " + - "SELECT * FROM s1 JOIN (SELECT * FROM s1, s2)") - - val commonSubqueries2 = df2.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child - }.distinct - assert(commonSubqueries2.length == 1) - - val df3 = - sql("WITH t1 AS (SELECT 1 AS id FROM t) SELECT * FROM t1 a, t1 b WHERE a.id = 1 AND b.id > 0") - - val commonSubqueries3 = df3.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child - }.distinct - assert(commonSubqueries3.length == 1) + withTempView("dedup") { + spark.range(10).createOrReplaceTempView("dedup") + val df = sql("WITH s AS (SELECT 1 FROM dedup) SELECT * FROM s s1 join s s2") + + val commonSubqueries = df.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries.length == 1) + + val df2 = sql("WITH s1 AS (SELECT 1 FROM dedup), s2 AS (SELECT 1 FROM dedup) " + + "SELECT * FROM s1 JOIN (SELECT * FROM s1, s2)") + + val commonSubqueries2 = df2.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries2.length == 1) + + val df3 = sql("WITH t1 AS (SELECT 1 AS id FROM dedup) SELECT * FROM t1 a, t1 b " + + "WHERE a.id = 1 AND b.id > 0") + + val commonSubqueries3 = df3.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries3.length == 1) + } // Using a self-join as CTE to test if de-duplicated attributes work for this. val df4 = sql("WITH j AS (SELECT * FROM (SELECT * FROM l JOIN l)) SELECT * FROM j j1, j j2") @@ -605,6 +607,10 @@ class SubquerySuite extends QueryTest with SharedSQLContext { case c: CommonSubqueryExec => c.subquery.child }.distinct assert(commonSubqueries4.length == 1) + + val df4WithoutCTE = sql("SELECT * FROM (SELECT * FROM (SELECT * FROM l JOIN l)) j1, " + + "(SELECT * FROM (SELECT * FROM l JOIN l)) j2") + assert(df4.collect() === df4WithoutCTE.collect()) } test("SPARK-16804: Correlated subqueries containing LIMIT - 1") { From 5282de7143daf4fb4086cad834f904b9eb8e088d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 12 Aug 2016 11:50:51 +0800 Subject: [PATCH 07/28] Fix some bugs. --- .../sql/catalyst/analysis/Analyzer.scala | 38 +++++++++++++++++++ .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 16 ++++++++ .../org/apache/spark/sql/SubquerySuite.scala | 8 ++++ 4 files changed, 63 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index aa8c9da8c1499..ce103b0f25aac 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -81,6 +81,8 @@ class Analyzer( val extendedResolutionRules: Seq[Rule[LogicalPlan]] = Nil lazy val batches: Seq[Batch] = Seq( + Batch("CTESubqueryAnalysis", Once, + CTESubqueryAnalysis), Batch("Substitution", fixedPoint, CTESubstitution, WindowsSubstitution, @@ -119,6 +121,42 @@ class Analyzer( CleanupAliases) ) + /** + * Analyses subqueries in CTE. + * In order to de-duplicate common subqueries in CTE, the underlying plans in the common + * subqueries should have the same attributes (i.e., no differences of expression IDs). To + * achieve this, we have to analyse the subqueries' parsed logical plans in advance, instead + * of just interpolating the parsed logical plans and analysing them individually. Because + * that will generate different attributes. + */ + object CTESubqueryAnalysis extends Rule[LogicalPlan] { + def substitutePreviousCTE( + plan: LogicalPlan, + ctes: Seq[(String, CommonSubqueryAlias)]): LogicalPlan = { + plan transformDown { + case u: UnresolvedRelation => + val substituted = ctes.find(x => resolver(x._1, u.tableIdentifier.table)) + .map(_._2).map { relation => + val withAlias = u.alias.map(CommonSubqueryAlias(_, relation.child)) + withAlias.getOrElse(relation) + } + substituted.getOrElse(u) + } + } + + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case With(child, relations) => + val analyzedRelations = relations.foldLeft(Seq.empty[(String, CommonSubqueryAlias)]) { + case (analyzed, (name, relation)) => + val newRelation = CommonSubqueryAlias(relation.alias, + execute(substitutePreviousCTE(relation.child, analyzed))) + analyzed :+ name -> newRelation + } + With(child, analyzedRelations.toMap) + case o => o + } + } + /** * Substitute child plan with cte definitions */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 180f6828fbf75..7dfaefd8db1f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -2065,7 +2065,7 @@ object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { val noRecursiveSubqueryPlan = plan.transformDown { // Eliminate the recursive subqueries which have the same output. case s @ CommonSubqueryAlias(_, child) - if child.find(p => p.isInstanceOf[SubqueryAlias] && p.sameResult(s)).isDefined => + if child.find(p => p.isInstanceOf[CommonSubqueryAlias] && p.sameResult(s)).isDefined => child } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 45f8514f8e32f..dc9b19ce2abfd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -698,6 +698,22 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNo case class CommonSubqueryAlias(alias: String, child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output.map(_.withQualifier(Some(alias))) + + override def sameResult(plan: LogicalPlan): Boolean = plan match { + case c: CommonSubqueryAlias => + val thisChild = child.collectFirst { + case p: LogicalPlan if !p.isInstanceOf[CommonSubqueryAlias] => p + } + val otherChild = c.child.collectFirst { + case p: LogicalPlan if !p.isInstanceOf[CommonSubqueryAlias] => p + } + if (thisChild.isDefined && otherChild.isDefined) { + thisChild.get.sameResult(otherChild.get) + } else { + false + } + case o => child.sameResult(o) + } } case class SubqueryAlias(alias: String, child: LogicalPlan) extends UnaryNode { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 9602b206930c6..52bda5cacdbf4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -611,6 +611,14 @@ class SubquerySuite extends QueryTest with SharedSQLContext { val df4WithoutCTE = sql("SELECT * FROM (SELECT * FROM (SELECT * FROM l JOIN l)) j1, " + "(SELECT * FROM (SELECT * FROM l JOIN l)) j2") assert(df4.collect() === df4WithoutCTE.collect()) + + // CTE subquery refers to previous CTE subquery. + val df5 = sql("WITH cte AS (SELECT * FROM l a, l b), cte2 AS (SELECT * FROM cte j1, cte) " + + "SELECT * FROM cte2 j3, cte j4") + val commonSubqueries5 = df5.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries5.length == 1) } test("SPARK-16804: Correlated subqueries containing LIMIT - 1") { From 9fe1fbe7f0690290f0ccceabb77dbb9198930cc4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 12 Aug 2016 12:53:34 +0800 Subject: [PATCH 08/28] Reuse all subqueries, instead of CTE subqueries. --- .../spark/sql/catalyst/analysis/Analyzer.scala | 10 +++++----- .../spark/sql/catalyst/optimizer/Optimizer.scala | 10 +++++----- .../spark/sql/catalyst/parser/AstBuilder.scala | 4 ++-- .../plans/logical/basicLogicalOperators.scala | 15 +++++---------- .../sql/catalyst/parser/PlanParserSuite.scala | 2 +- .../apache/spark/sql/catalyst/SQLBuilder.scala | 2 -- .../org/apache/spark/sql/execution/subquery.scala | 4 ++-- 7 files changed, 20 insertions(+), 27 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 84c0a5b920bd9..dcba2aa29f4c7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -132,12 +132,12 @@ class Analyzer( object CTESubqueryAnalysis extends Rule[LogicalPlan] { def substitutePreviousCTE( plan: LogicalPlan, - ctes: Seq[(String, CommonSubqueryAlias)]): LogicalPlan = { + ctes: Seq[(String, SubqueryAlias)]): LogicalPlan = { plan transformDown { case u: UnresolvedRelation => val substituted = ctes.find(x => resolver(x._1, u.tableIdentifier.table)) .map(_._2).map { relation => - val withAlias = u.alias.map(CommonSubqueryAlias(_, relation.child)) + val withAlias = u.alias.map(SubqueryAlias(_, relation.child)) withAlias.getOrElse(relation) } substituted.getOrElse(u) @@ -146,9 +146,9 @@ class Analyzer( def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case With(child, relations) => - val analyzedRelations = relations.foldLeft(Seq.empty[(String, CommonSubqueryAlias)]) { + val analyzedRelations = relations.foldLeft(Seq.empty[(String, SubqueryAlias)]) { case (analyzed, (name, relation)) => - val newRelation = CommonSubqueryAlias(relation.alias, + val newRelation = SubqueryAlias(relation.alias, execute(substitutePreviousCTE(relation.child, analyzed))) analyzed :+ name -> newRelation } @@ -176,7 +176,7 @@ class Analyzer( // see https://github.com/apache/spark/pull/4929#discussion_r27186638 for more info case u : UnresolvedRelation => val substituted = cteRelations.get(u.tableIdentifier.table).map { relation => - val withAlias = u.alias.map(CommonSubqueryAlias(_, relation)) + val withAlias = u.alias.map(SubqueryAlias(_, relation)) withAlias.getOrElse(relation) } substituted.getOrElse(u) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 7dfaefd8db1f5..4c9f49c2df5e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -2055,7 +2055,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { } /** - * Removes the [[CommonSubqueryAlias]] operators which are used only once from the plan. + * Removes the [[SubqueryAlias]] operators which are used only once from the plan. */ object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { @@ -2064,14 +2064,14 @@ object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { val noRecursiveSubqueryPlan = plan.transformDown { // Eliminate the recursive subqueries which have the same output. - case s @ CommonSubqueryAlias(_, child) - if child.find(p => p.isInstanceOf[CommonSubqueryAlias] && p.sameResult(s)).isDefined => + case s @ SubqueryAlias(_, child) + if child.find(p => p.isInstanceOf[SubqueryAlias] && p.sameResult(s)).isDefined => child } noRecursiveSubqueryPlan.foreach { // Collect the subqueries that are used more than once in the query. - case CommonSubqueryAlias(_, child) => + case SubqueryAlias(_, child) => if (subqueries.indexWhere(s => s.sameResult(child)) >= 0) { // If the plan with same results can be found. duplicateSubqueries += child @@ -2084,7 +2084,7 @@ object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { noRecursiveSubqueryPlan.transformDown { // Only eliminate the subqueries that are used only once in the query. - case CommonSubqueryAlias(_, child) + case SubqueryAlias(_, child) if duplicateSubqueries.indexWhere(s => s.sameResult(child)) < 0 => child } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 97cf6af879042..c7fdc287d1995 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -106,8 +106,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * * This is only used for Common Table Expressions. */ - override def visitNamedQuery(ctx: NamedQueryContext): CommonSubqueryAlias = withOrigin(ctx) { - CommonSubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith)) + override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) { + SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith)) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index dc9b19ce2abfd..00f340089858d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -396,7 +396,7 @@ case class InsertIntoTable( * key is the alias of the CTE definition, * value is the CTE definition. */ -case class With(child: LogicalPlan, cteRelations: Map[String, CommonSubqueryAlias]) +case class With(child: LogicalPlan, cteRelations: Map[String, SubqueryAlias]) extends UnaryNode { override def output: Seq[Attribute] = child.output } @@ -695,17 +695,17 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNo } } -case class CommonSubqueryAlias(alias: String, child: LogicalPlan) extends UnaryNode { +case class SubqueryAlias(alias: String, child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output.map(_.withQualifier(Some(alias))) override def sameResult(plan: LogicalPlan): Boolean = plan match { - case c: CommonSubqueryAlias => + case c: SubqueryAlias => val thisChild = child.collectFirst { - case p: LogicalPlan if !p.isInstanceOf[CommonSubqueryAlias] => p + case p: LogicalPlan if !p.isInstanceOf[SubqueryAlias] => p } val otherChild = c.child.collectFirst { - case p: LogicalPlan if !p.isInstanceOf[CommonSubqueryAlias] => p + case p: LogicalPlan if !p.isInstanceOf[SubqueryAlias] => p } if (thisChild.isDefined && otherChild.isDefined) { thisChild.get.sameResult(otherChild.get) @@ -716,11 +716,6 @@ case class CommonSubqueryAlias(alias: String, child: LogicalPlan) extends UnaryN } } -case class SubqueryAlias(alias: String, child: LogicalPlan) extends UnaryNode { - - override def output: Seq[Attribute] = child.output.map(_.withQualifier(Some(alias))) -} - /** * Sample the dataset. * diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index f74046bdb7991..00a37cf6360ae 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -80,7 +80,7 @@ class PlanParserSuite extends PlanTest { def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = { val ctes = namedPlans.map { case (name, cte) => - name -> CommonSubqueryAlias(name, cte) + name -> SubqueryAlias(name, cte) }.toMap With(plan, ctes) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index 415cf98e2738c..5d93419f357ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -164,8 +164,6 @@ class SQLBuilder private ( case p: SubqueryAlias => build("(" + toSQL(p.child) + ")", "AS", p.alias) - case p: CommonSubqueryAlias => build("(" + toSQL(p.child) + ")", "AS", p.alias) - case p: Join => build( toSQL(p.left), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index b443f4d7aece7..7b0c5280ae9a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.catalyst.plans.logical.{CommonSubqueryAlias, Filter, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, SubqueryAlias} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.subquery.SubqueryDedup import org.apache.spark.sql.internal.SQLConf @@ -201,7 +201,7 @@ case class DedupCommonSubqueries(sparkSession: SparkSession) extends Rule[Logica def apply(plan: LogicalPlan): LogicalPlan = { val dedup = new SubqueryDedup() plan.transformDown { - case s: CommonSubqueryAlias => + case s: SubqueryAlias => val optimized = sparkSession.sessionState.optimizer.execute(s.child) dedup.createCommonSubquery(sparkSession, optimized) } From a14459c8d4245be6e89d65593d737d250412d01f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 12 Aug 2016 15:11:39 +0800 Subject: [PATCH 09/28] Can not use optimizedPlan when replacing common subquery because we need to keep analyzed plan's output. --- .../main/scala/org/apache/spark/sql/execution/subquery.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 7b0c5280ae9a9..7625b8055c1a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -202,8 +202,7 @@ case class DedupCommonSubqueries(sparkSession: SparkSession) extends Rule[Logica val dedup = new SubqueryDedup() plan.transformDown { case s: SubqueryAlias => - val optimized = sparkSession.sessionState.optimizer.execute(s.child) - dedup.createCommonSubquery(sparkSession, optimized) + dedup.createCommonSubquery(sparkSession, s.child) } } } From 9d7a15df4d0c78d5e2d68b2e56c605de0b4bee4d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 18 Aug 2016 10:00:16 +0800 Subject: [PATCH 10/28] Optimization of common subqueries. --- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 156 ++++++++++++++++-- .../plans/logical/basicLogicalOperators.scala | 12 +- .../spark/sql/catalyst/SQLBuilder.scala | 2 +- .../spark/sql/execution/QueryExecution.scala | 8 +- .../apache/spark/sql/execution/subquery.scala | 4 +- .../org/apache/spark/sql/SubquerySuite.scala | 48 ++++++ 8 files changed, 213 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index b5bd2635d0b5c..a1ffc138be9f4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2096,7 +2096,7 @@ class Analyzer( */ object EliminateSubqueryAliases extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case SubqueryAlias(_, child) => child + case SubqueryAlias(_, child, _) => child } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 41b7e62d8ccea..1fc6800752b8e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -141,7 +141,7 @@ trait CheckAnalysis extends PredicateHelper { // Skip projects and subquery aliases added by the Analyzer and the SQLBuilder. def cleanQuery(p: LogicalPlan): LogicalPlan = p match { - case SubqueryAlias(_, child) => cleanQuery(child) + case SubqueryAlias(_, child, _) => cleanQuery(child) case Project(_, child) => cleanQuery(child) case child => child } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 4c9f49c2df5e8..acd974837b4f0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.annotation.tailrec import scala.collection.immutable.HashSet import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap import org.apache.spark.api.java.function.FilterFunction import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf} @@ -49,7 +50,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) // However, because we also use the analyzer to canonicalized queries (for view definition), // we do not eliminate subqueries or compute current time in the analyzer. Batch("Finish Analysis", Once, - EliminateSubqueryAliases, + EliminateOneTimeSubqueryAliases, ReplaceExpressions, ComputeCurrentTime, GetCurrentDatabase(sessionCatalog), @@ -464,6 +465,9 @@ object ColumnPruning extends Rule[LogicalPlan] { // Can't prune the columns on LeafNode case p @ Project(_, _: LeafNode) => p + // Don't prune the columns on common subquery. + case p @ Project(_, SubqueryAlias(_, _, true)) => p + // for all other logical plans that inherits the output from it's children case p @ Project(_, child) => val required = child.references ++ p.references @@ -1210,6 +1214,8 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { case filter @ Filter(_, _: Filter) => filter // should not push predicates through sample, or will generate different results. case filter @ Filter(_, _: Sample) => filter + // should not push predicates through common subquery. + case filter @ Filter(_, SubqueryAlias(_, _, true)) => filter case filter @ Filter(condition, u: UnaryNode) if u.expressions.forall(_.deterministic) => pushDownPredicate(filter, u.child) { predicate => @@ -1862,7 +1868,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { // and Project operators, followed by an optional Filter, followed by an // Aggregate. Traverse the operators recursively. def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = lp match { - case SubqueryAlias(_, child) => evalPlan(child) + case SubqueryAlias(_, child, false) => evalPlan(child) case Filter(condition, child) => val bindings = evalPlan(child) if (bindings.isEmpty) bindings @@ -1920,7 +1926,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { topPart += p bottomPart = child - case s @ SubqueryAlias(_, child) => + case s @ SubqueryAlias(_, child, false) => topPart += s bottomPart = child @@ -1991,7 +1997,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { topPart.reverse.foreach { case Project(projList, _) => subqueryRoot = Project(projList ++ havingInputs, subqueryRoot) - case s @ SubqueryAlias(alias, _) => + case s @ SubqueryAlias(alias, _, false) => subqueryRoot = SubqueryAlias(alias, subqueryRoot) case op => sys.error(s"Unexpected operator $op in corelated subquery") } @@ -2054,6 +2060,130 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { } } +/** + * Optimizes the logical plans wrapped in SubqueryAlias and operators on them. + * The SubqueryAlias which are remaining in optimization phase are common subqueries, + * i.e., they are duplicate in the whole query plan. The logical plans wrapped in + * SubqueryAlias will be executed individually later. However, some operators such as + * Project and Filter can be optimized with the wrapped logical plans. Thus, this rule + * considers the optimization of the wrapped logical plans and operators on SubqueryAlias. + */ +case class OptimizeCommonSubqueries(optimizer: Optimizer) extends Rule[LogicalPlan] { + // Optimized the subqueries which all have a Project parent node and the same results. + private def optimizeProjectWithSubqueries( + plan: LogicalPlan, + subqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = { + plan transform { + case p @ Project(pList, SubqueryAlias(alias, subquery, true)) => + val pListForAll: Seq[NamedExpression] = subqueries.flatMap { case Project(pList, child) => + val rewrites = buildRewrites(child, subquery) + pList.map(pushToOtherPlan(_, rewrites)) + } + + val newSubquery = Project(pListForAll, subquery) + val optimized = optimizer.execute(newSubquery) + // Check if any optimization is performed. + if (optimized.sameResult(newSubquery)) { + // No optimization happens. Let's keep original subquery. + p + } else { + Project(pList, SubqueryAlias(alias, newSubquery, true)) + } + } + } + + /** + * Maps Attributes from the source side to the corresponding Attribute on the target side. + */ + private def buildRewrites(source: LogicalPlan, target: LogicalPlan): AttributeMap[Attribute] = { + assert(source.output.size == target.output.size) + AttributeMap(source.output.zip(target.output)) + } + + /** + * Rewrites an expression so that it can be pushed to another LogicalPlan. + */ + private def pushToOtherPlan[A <: Expression](e: A, rewrites: AttributeMap[Attribute]) = { + val result = e transform { + case a: Attribute => rewrites(a) + } + + // We must promise the compiler that we did not discard the names in the case of project + // expressions. This is safe since the only transformation is from Attribute => Attribute. + result.asInstanceOf[A] + } + + private def optimizeFilterWithSubqueries( + plan: LogicalPlan, + subqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = { + plan transform { + case f @ Filter(cond, SubqueryAlias(alias, subquery, true)) => + val conditionForAll: Expression = subqueries.map { case Filter(otherCond, child) => + val rewrites = buildRewrites(child, subquery) + pushToOtherPlan(otherCond, rewrites) + }.reduce(Or) + + val newSubquery = Filter(conditionForAll, subquery) + val optimized = optimizer.execute(newSubquery) + // Check if any optimization is performed. + if (optimized.sameResult(newSubquery)) { + // No optimization happens. Let's keep original subquery. + f + } else { + Filter(cond, SubqueryAlias(alias, newSubquery, true)) + } + } + } + + def apply(plan: LogicalPlan): LogicalPlan = { + val subqueryMap = HashMap.empty[LogicalPlan, ArrayBuffer[LogicalPlan]] + + // Constructs the groups of the subqueries with the same results. + plan.foreach { + case u: UnaryNode + if u.child.isInstanceOf[SubqueryAlias] && + u.child.asInstanceOf[SubqueryAlias].commonSubquery => + + val child = u.child.asInstanceOf[SubqueryAlias].child + // Looking for the existing group with the same results. + subqueryMap.find { case (key, _) => + if (key.sameResult(child)) { + true + } else { + false + } + }.map { case (_, subqueries) => + // If found, add current logical plan into this group. + subqueries += u + }.getOrElse { + // If not, create a new group. + subqueryMap += ((child, ArrayBuffer[LogicalPlan](u))) + } + case _ => + } + + // Begins to optimize common SubqueryAlias with outside operators. + // We only need to take care two cases: + // 1. All subqueries have a Project on them. + // 2. All subqueries have a Filter on them. + var currentPlan = plan + subqueryMap.map { case (_, subqueries) => + if (subqueries.length > 1) { + val allProject = subqueries.forall(_.isInstanceOf[Project]) + if (allProject) { + currentPlan = optimizeProjectWithSubqueries(currentPlan, subqueries) + } else { + val allFilter = subqueries.forall(_.isInstanceOf[Filter]) + if (allFilter) { + currentPlan = optimizeFilterWithSubqueries(currentPlan, subqueries) + } + } + } + } + currentPlan + } +} + /** * Removes the [[SubqueryAlias]] operators which are used only once from the plan. */ @@ -2064,14 +2194,14 @@ object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { val noRecursiveSubqueryPlan = plan.transformDown { // Eliminate the recursive subqueries which have the same output. - case s @ SubqueryAlias(_, child) + case s @ SubqueryAlias(_, child, _) if child.find(p => p.isInstanceOf[SubqueryAlias] && p.sameResult(s)).isDefined => child } noRecursiveSubqueryPlan.foreach { - // Collect the subqueries that are used more than once in the query. - case SubqueryAlias(_, child) => + // Collects the subqueries that are used more than once in the query. + case SubqueryAlias(_, child, _) => if (subqueries.indexWhere(s => s.sameResult(child)) >= 0) { // If the plan with same results can be found. duplicateSubqueries += child @@ -2082,11 +2212,15 @@ object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { case _ => } + // Set the `commonSubquery` of remainning SubqueryAlias as `true`. noRecursiveSubqueryPlan.transformDown { - // Only eliminate the subqueries that are used only once in the query. - case SubqueryAlias(_, child) - if duplicateSubqueries.indexWhere(s => s.sameResult(child)) < 0 => - child + // Eliminates the subqueries that are used only once in the query. + case SubqueryAlias(alias, child, _) => + if (duplicateSubqueries.indexWhere(s => s.sameResult(child)) < 0) { + child + } else { + SubqueryAlias(alias, child, commonSubquery = true) + } } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 801cc4694df39..455b14efae526 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -693,7 +693,17 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNo } } -case class SubqueryAlias(alias: String, child: LogicalPlan) extends UnaryNode { +/** + * Represents a subquery in query plan. + * @param alias The name of this subquery. + * @param child The logical plan of the subquery. + * @param commonSubquery Whether this subquery is a common subquery, i.e., the logical plan + * referred more than once in the query plan. Default: false. + */ +case class SubqueryAlias( + alias: String, + child: LogicalPlan, + commonSubquery: Boolean = false) extends UnaryNode { override def output: Seq[Attribute] = child.output.map(_.withQualifier(Some(alias))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index 5d93419f357ef..9ea644a303a80 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -440,7 +440,7 @@ class SQLBuilder private ( object RemoveSubqueriesAboveSQLTable extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case SubqueryAlias(_, t @ ExtractSQLTable(_)) => t + case SubqueryAlias(_, t @ ExtractSQLTable(_), _) => t } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index a685efc2dfd86..4a4f7f41e7f76 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -24,7 +24,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker -import org.apache.spark.sql.catalyst.optimizer.EliminateOneTimeSubqueryAliases +import org.apache.spark.sql.catalyst.optimizer.OptimizeCommonSubqueries import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -73,9 +73,9 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { } lazy val optimizedPlan: LogicalPlan = { - val noOneTimeCommonSubqueries = EliminateOneTimeSubqueryAliases(withCachedData) - val dedupSubqueries = DedupCommonSubqueries(sparkSession)(noOneTimeCommonSubqueries) - sparkSession.sessionState.optimizer.execute(dedupSubqueries) + val optimized = sparkSession.sessionState.optimizer.execute(withCachedData) + val subqueryOptimized = OptimizeCommonSubqueries(sparkSession.sessionState.optimizer)(optimized) + DedupCommonSubqueries(sparkSession)(subqueryOptimized) } lazy val sparkPlan: SparkPlan = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 7625b8055c1a8..f04d0b21244ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -201,8 +201,8 @@ case class DedupCommonSubqueries(sparkSession: SparkSession) extends Rule[Logica def apply(plan: LogicalPlan): LogicalPlan = { val dedup = new SubqueryDedup() plan.transformDown { - case s: SubqueryAlias => - dedup.createCommonSubquery(sparkSession, s.child) + case s @ SubqueryAlias(_, child, true) => + dedup.createCommonSubquery(sparkSession, child) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 52bda5cacdbf4..233d6757d7449 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -17,9 +17,12 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} +import org.apache.spark.sql.execution.{FilterExec, ProjectExec} import org.apache.spark.sql.execution.subquery.CommonSubqueryExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{DoubleType, IntegerType} class SubquerySuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -621,6 +624,51 @@ class SubquerySuite extends QueryTest with SharedSQLContext { assert(commonSubqueries5.length == 1) } + test("Dedup subqueries with optimization: Filter pushdown") { + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + val df = sql("WITH cte AS (SELECT a.a AS a, a.b AS b, b.a AS c, b.b AS d FROM l a, l b) " + + "SELECT * FROM (SELECT * FROM cte WHERE a = 1) x JOIN (SELECT * FROM cte WHERE b = 1.0) y") + + val commonSubqueries = df.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries.length == 1) + val pushdownFilter = commonSubqueries(0).collect { + case f: FilterExec => f + } + assert(pushdownFilter.length == 1) + val intConditions = pushdownFilter(0).asInstanceOf[FilterExec].condition.collect { + case EqualTo(a: AttributeReference, Literal(i, IntegerType)) => (a.name, i) + } + assert(intConditions.length == 1 && intConditions(0)._1 == "a" && intConditions(0)._2 == 1) + val doubleConditions = pushdownFilter(0).asInstanceOf[FilterExec].condition.collect { + case EqualTo(a: AttributeReference, Literal(d, DoubleType)) => (a.name, d) + } + assert(doubleConditions.length == 1 && + doubleConditions(0)._1 == "b" && doubleConditions(0)._2 == 1.0) + } + } + + test("Dedup subqueries with optimization: Project pushdown") { + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + val df = sql("WITH cte AS (SELECT a.a AS a, a.b AS b, b.a AS c, b.b AS d FROM l a, l b) " + + "SELECT * FROM (SELECT a FROM cte) x JOIN (SELECT b FROM cte) y") + + val commonSubqueries = df.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries.length == 1) + + val pushdownProject = commonSubqueries(0).collect { + case f: ProjectExec => f + } + assert(pushdownProject.length == 1) + // As we project `a` and `b` which both are located at only one side at the inner join, + // optimizer will add an empty projection node to filter out all rows from other side. + assert(pushdownProject(0).asInstanceOf[ProjectExec].projectList.isEmpty) + } + } + test("SPARK-16804: Correlated subqueries containing LIMIT - 1") { withTempView("onerow") { Seq(1).toDF("c1").createOrReplaceTempView("onerow") From f3aa0aa41f9c1c256053e108226e3e92c07e6ee5 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Aug 2016 10:51:36 +0800 Subject: [PATCH 11/28] Fix a bug and add test. --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 6 ++++++ .../scala/org/apache/spark/sql/SubquerySuite.scala | 11 +++++++++++ 2 files changed, 17 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 8c4b9512a1eff..b871aeb84d517 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -142,6 +142,12 @@ class Analyzer( withAlias.getOrElse(relation) } substituted.getOrElse(u) + case other => + // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. + other transformExpressions { + case e: SubqueryExpression => + e.withNewPlan(substitutePreviousCTE(e.plan, ctes)) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 233d6757d7449..9e4d1e1769119 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -649,6 +649,17 @@ class SubquerySuite extends QueryTest with SharedSQLContext { } } + test("Dedup subqueries with uncorrelated scalar subquery in CTE") { + val df = sql("WITH t1 AS (SELECT 1 AS b, 2 AS c), " + + "t2 AS (SELECT a FROM (SELECT 1 AS a UNION ALL SELECT 2 AS a) t " + + "WHERE a = (SELECT max(b) FROM t1)) SELECT * FROM t2 x UNION ALL SELECT * FROM t2 y") + checkAnswer(df, Array(Row(1), Row(1))) + val commonSubqueries = df.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries.length == 1) + } + test("Dedup subqueries with optimization: Project pushdown") { withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { val df = sql("WITH cte AS (SELECT a.a AS a, a.b AS b, b.a AS c, b.b AS d FROM l a, l b) " + From e094c141cf9573fa09123a657ae2a4b786b3725c Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Aug 2016 11:53:38 +0800 Subject: [PATCH 12/28] Pushdowned projection list should use attributes. --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index e54203d89b645..c2c032b5521a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -2074,9 +2074,11 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) extends Rule[LogicalPl // Optimized the subqueries which all have a Project parent node and the same results. private def optimizeProjectWithSubqueries( plan: LogicalPlan, + keyPlan: LogicalPlan, subqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = { plan transform { - case p @ Project(pList, SubqueryAlias(alias, subquery, v, true)) => + case p @ Project(pList, s @ SubqueryAlias(alias, subquery, v, true)) + if s.sameResult(keyPlan) => val pListForAll: Seq[NamedExpression] = subqueries.flatMap { case Project(pList, child) => val rewrites = buildRewrites(child, subquery) pList.map(pushToOtherPlan(_, rewrites)) @@ -2089,7 +2091,7 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) extends Rule[LogicalPl // No optimization happens. Let's keep original subquery. p } else { - Project(pList, SubqueryAlias(alias, newSubquery, v, true)) + Project(pList.map(_.toAttribute), SubqueryAlias(alias, newSubquery, v, true)) } } } @@ -2117,9 +2119,10 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) extends Rule[LogicalPl private def optimizeFilterWithSubqueries( plan: LogicalPlan, + keyPlan: LogicalPlan, subqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = { plan transform { - case f @ Filter(cond, SubqueryAlias(alias, subquery, v, true)) => + case f @ Filter(cond, s @ SubqueryAlias(alias, subquery, v, true)) if s.sameResult(keyPlan) => val conditionForAll: Expression = subqueries.map { case Filter(otherCond, child) => val rewrites = buildRewrites(child, subquery) pushToOtherPlan(otherCond, rewrites) @@ -2169,15 +2172,15 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) extends Rule[LogicalPl // 1. All subqueries have a Project on them. // 2. All subqueries have a Filter on them. var currentPlan = plan - subqueryMap.map { case (_, subqueries) => + subqueryMap.map { case (key, subqueries) => if (subqueries.length > 1) { val allProject = subqueries.forall(_.isInstanceOf[Project]) if (allProject) { - currentPlan = optimizeProjectWithSubqueries(currentPlan, subqueries) + currentPlan = optimizeProjectWithSubqueries(currentPlan, key, subqueries) } else { val allFilter = subqueries.forall(_.isInstanceOf[Filter]) if (allFilter) { - currentPlan = optimizeFilterWithSubqueries(currentPlan, subqueries) + currentPlan = optimizeFilterWithSubqueries(currentPlan, key, subqueries) } } } From f0954cddbf80b65f1e0aac694be97b4bf9e29436 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 23 Aug 2016 14:51:58 +0800 Subject: [PATCH 13/28] Fix a bug in filter pushdown. --- .../sql/catalyst/optimizer/Optimizer.scala | 36 +++++++++++++++---- .../org/apache/spark/sql/SubquerySuite.scala | 21 +++++++++++ 2 files changed, 50 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 54b34e72f5e2a..d576fffa61843 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -2079,7 +2079,8 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { * Project and Filter can be optimized with the wrapped logical plans. Thus, this rule * considers the optimization of the wrapped logical plans and operators on SubqueryAlias. */ -case class OptimizeCommonSubqueries(optimizer: Optimizer) extends Rule[LogicalPlan] { +case class OptimizeCommonSubqueries(optimizer: Optimizer) + extends Rule[LogicalPlan] with PredicateHelper { // Optimized the subqueries which all have a Project parent node and the same results. private def optimizeProjectWithSubqueries( plan: LogicalPlan, @@ -2118,7 +2119,7 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) extends Rule[LogicalPl */ private def pushToOtherPlan[A <: Expression](e: A, rewrites: AttributeMap[Attribute]) = { val result = e transform { - case a: Attribute => rewrites(a) + case a: Attribute => rewrites.get(a).getOrElse(a) } // We must promise the compiler that we did not discard the names in the case of project @@ -2130,14 +2131,35 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) extends Rule[LogicalPl plan: LogicalPlan, keyPlan: LogicalPlan, subqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = { + val firstCondExprs = splitConjunctivePredicates(subqueries(0).asInstanceOf[Filter].condition) + val firstPushdownCondition: Expression = (Seq(subqueries(0).asInstanceOf[Filter].condition) ++ + subqueries.tail.flatMap { + case Filter(otherCond, child) => + val rewrites = buildRewrites(child, subqueries(0).asInstanceOf[Filter].child) + val newCond = pushToOtherPlan(otherCond, rewrites) + val condExprs = splitConjunctivePredicates(newCond) + val common = firstCondExprs.filter(e => condExprs.exists(e.semanticEquals)) + if (common.isEmpty) { + Seq(newCond) + } else { + val diff = condExprs.filterNot(e => common.exists(e.semanticEquals)) + if (!diff.isEmpty) { + Seq(diff.reduce(And)) + } else { + Seq() + } + } + }).reduce(Or) + plan transform { case f @ Filter(cond, s @ SubqueryAlias(alias, subquery, v, true)) if s.sameResult(keyPlan) => - val conditionForAll: Expression = subqueries.map { case Filter(otherCond, child) => - val rewrites = buildRewrites(child, subquery) - pushToOtherPlan(otherCond, rewrites) - }.reduce(Or) + val pushdownCond: Expression = subqueries.foldLeft(firstPushdownCondition) { + case (currentCond, sub) => + val rewrites = buildRewrites(sub.asInstanceOf[Filter].child, subquery) + pushToOtherPlan(currentCond, rewrites) + } - val newSubquery = Filter(conditionForAll, subquery) + val newSubquery = Filter(pushdownCond, subquery) val optimized = optimizer.execute(newSubquery) // Check if any optimization is performed. if (optimized.sameResult(newSubquery)) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 9e4d1e1769119..fb62bf64f8cdb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -646,6 +646,27 @@ class SubquerySuite extends QueryTest with SharedSQLContext { } assert(doubleConditions.length == 1 && doubleConditions(0)._1 == "b" && doubleConditions(0)._2 == 1.0) + + val df2 = sql("WITH cte AS (SELECT a.a AS a, a.b AS b, b.a AS c, b.b AS d FROM l a, l b) " + + "SELECT * FROM cte x, cte y WHERE x.b = y.b AND x.a = 1 AND x.b = 1 + 1") + + val commonSubqueries2 = df2.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries2.length == 1) + val pushdownFilter2 = commonSubqueries2(0).collect { + case f: FilterExec => f + } + assert(pushdownFilter2.length == 1) + val intConditions2 = pushdownFilter2(0).asInstanceOf[FilterExec].condition.collect { + case EqualTo(a: AttributeReference, Literal(i, IntegerType)) => (a.name, i) + } + assert(intConditions2.length == 1 && intConditions2(0)._1 == "a" && intConditions2(0)._2 == 1) + val doubleConditions2 = pushdownFilter2(0).asInstanceOf[FilterExec].condition.collect { + case EqualTo(a: AttributeReference, Literal(d, DoubleType)) => (a.name, d) + } + assert(doubleConditions2.length == 1 && + doubleConditions2(0)._1 == "b" && doubleConditions2(0)._2 == 2.0) } } From 6a8011bc9dfa3289e98a5efe65e92704b85bb4b5 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 25 Aug 2016 21:39:37 +0800 Subject: [PATCH 14/28] Fix filter pushdown. --- .../sql/catalyst/optimizer/Optimizer.scala | 20 ++++--------------- .../org/apache/spark/sql/SubquerySuite.scala | 6 +++++- 2 files changed, 9 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index d576fffa61843..ac183c0b711e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -2132,23 +2132,11 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) keyPlan: LogicalPlan, subqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = { val firstCondExprs = splitConjunctivePredicates(subqueries(0).asInstanceOf[Filter].condition) - val firstPushdownCondition: Expression = (Seq(subqueries(0).asInstanceOf[Filter].condition) ++ - subqueries.tail.flatMap { + val firstPushdownCondition: Expression = (subqueries(0).asInstanceOf[Filter].condition +: + subqueries.tail.map { case Filter(otherCond, child) => val rewrites = buildRewrites(child, subqueries(0).asInstanceOf[Filter].child) - val newCond = pushToOtherPlan(otherCond, rewrites) - val condExprs = splitConjunctivePredicates(newCond) - val common = firstCondExprs.filter(e => condExprs.exists(e.semanticEquals)) - if (common.isEmpty) { - Seq(newCond) - } else { - val diff = condExprs.filterNot(e => common.exists(e.semanticEquals)) - if (!diff.isEmpty) { - Seq(diff.reduce(And)) - } else { - Seq() - } - } + pushToOtherPlan(otherCond, rewrites) }).reduce(Or) plan transform { @@ -2203,7 +2191,7 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) // 1. All subqueries have a Project on them. // 2. All subqueries have a Filter on them. var currentPlan = plan - subqueryMap.map { case (key, subqueries) => + subqueryMap.foreach { case (key, subqueries) => if (subqueries.length > 1) { val allProject = subqueries.forall(_.isInstanceOf[Project]) if (allProject) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index fb62bf64f8cdb..edfd22e62ff31 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -647,6 +647,10 @@ class SubquerySuite extends QueryTest with SharedSQLContext { assert(doubleConditions.length == 1 && doubleConditions(0)._1 == "b" && doubleConditions(0)._2 == 1.0) + // There are two Filters: + // 1. x.a = 1 && x.b = 2.0 + // 2. x.b = 2.0 + // The pushdown Filter is ((x.a = 1 && x.b = 2.0) || (x.b = 2.0)) => (x.b = 2.0). val df2 = sql("WITH cte AS (SELECT a.a AS a, a.b AS b, b.a AS c, b.b AS d FROM l a, l b) " + "SELECT * FROM cte x, cte y WHERE x.b = y.b AND x.a = 1 AND x.b = 1 + 1") @@ -661,7 +665,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { val intConditions2 = pushdownFilter2(0).asInstanceOf[FilterExec].condition.collect { case EqualTo(a: AttributeReference, Literal(i, IntegerType)) => (a.name, i) } - assert(intConditions2.length == 1 && intConditions2(0)._1 == "a" && intConditions2(0)._2 == 1) + assert(intConditions2.length == 0) val doubleConditions2 = pushdownFilter2(0).asInstanceOf[FilterExec].condition.collect { case EqualTo(a: AttributeReference, Literal(d, DoubleType)) => (a.name, d) } From 6cb40f12e074e0350aa01778c955b35631160858 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 25 Aug 2016 22:21:59 +0800 Subject: [PATCH 15/28] Skip duplicating table scan nodes. --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 3 ++- .../scala/org/apache/spark/sql/SubquerySuite.scala | 14 ++++++++------ 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index ac183c0b711e3..ceb8999f2a0f4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -2225,7 +2225,8 @@ object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { noRecursiveSubqueryPlan.foreach { // Collects the subqueries that are used more than once in the query. - case SubqueryAlias(_, child, _, _) => + // Skip the SubqueryAlias on table scan. + case SubqueryAlias(_, child, _, _) if !child.isInstanceOf[MultiInstanceRelation] => if (subqueries.indexWhere(s => s.sameResult(child)) >= 0) { // If the plan with same results can be found. duplicateSubqueries += child diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index edfd22e62ff31..bf9c8ebfa897b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} -import org.apache.spark.sql.execution.{FilterExec, ProjectExec} +import org.apache.spark.sql.execution.{FilterExec, LocalTableScanExec} import org.apache.spark.sql.execution.subquery.CommonSubqueryExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -695,13 +695,15 @@ class SubquerySuite extends QueryTest with SharedSQLContext { }.distinct assert(commonSubqueries.length == 1) - val pushdownProject = commonSubqueries(0).collect { - case f: ProjectExec => f + val localTableScan = commonSubqueries(0).collect { + case l: LocalTableScanExec => l } - assert(pushdownProject.length == 1) + // As we project `a` and `b` which both are located at only one side at the inner join, - // optimizer will add an empty projection node to filter out all rows from other side. - assert(pushdownProject(0).asInstanceOf[ProjectExec].projectList.isEmpty) + // one [[LocalTableScanExec]] will have empty output after column pruning. + assert(localTableScan.length == 2) + assert(localTableScan(0).asInstanceOf[LocalTableScanExec].output.isEmpty || + localTableScan(1).asInstanceOf[LocalTableScanExec].output.isEmpty) } } From 6a08486fe535547574d5fa3b278f2862488d5d08 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 29 Aug 2016 12:58:40 +0800 Subject: [PATCH 16/28] Don't do nested common subquery deduplication. --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index ceb8999f2a0f4..bc431065d2f0b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -2138,8 +2138,7 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) val rewrites = buildRewrites(child, subqueries(0).asInstanceOf[Filter].child) pushToOtherPlan(otherCond, rewrites) }).reduce(Or) - - plan transform { + plan transformDown { case f @ Filter(cond, s @ SubqueryAlias(alias, subquery, v, true)) if s.sameResult(keyPlan) => val pushdownCond: Expression = subqueries.foldLeft(firstPushdownCondition) { case (currentCond, sub) => @@ -2149,6 +2148,7 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) val newSubquery = Filter(pushdownCond, subquery) val optimized = optimizer.execute(newSubquery) + // Check if any optimization is performed. if (optimized.sameResult(newSubquery)) { // No optimization happens. Let's keep original subquery. @@ -2244,7 +2244,11 @@ object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { if (duplicateSubqueries.indexWhere(s => s.sameResult(child)) < 0) { child } else { - SubqueryAlias(alias, child, v, commonSubquery = true) + // Strip all wrapped subqueries. + val newChild = child.transformDown { + case s: SubqueryAlias => s.child + } + SubqueryAlias(alias, newChild, v, commonSubquery = true) } } } From c6d987f584859224a05b17a157be30389066dccb Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 29 Aug 2016 16:30:08 +0800 Subject: [PATCH 17/28] Change to common subquery query plans to make the query tree more reasonable. --- .../execution/subquery/CommonSubquery.scala | 13 ++++++------- .../subquery/CommonSubqueryExec.scala | 10 +++++----- .../sql/execution/subquery/SubqueryDedup.scala | 5 +++-- .../org/apache/spark/sql/SubquerySuite.scala | 18 +++++++++--------- 4 files changed, 23 insertions(+), 23 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala index a68210afbf529..b6d6eff254686 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala @@ -22,17 +22,16 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.plans.logical.Statistics +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.execution.SparkPlan private[sql] case class CommonSubquery( output: Seq[Attribute], - @transient child: SparkPlan)( + @transient child: LogicalPlan)( + @transient val executedChild: SparkPlan, private[sql] val _statistics: Statistics, @transient private[sql] var _computedOutput: RDD[InternalRow] = null) - extends logical.LeafNode { - - override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) + extends logical.UnaryNode { override def producedAttributes: AttributeSet = outputSet @@ -41,12 +40,12 @@ private[sql] case class CommonSubquery( lazy val numRows: Long = computedOutput.count def withOutput(newOutput: Seq[Attribute]): CommonSubquery = { - CommonSubquery(newOutput, child)(_statistics, _computedOutput) + CommonSubquery(newOutput, child)(executedChild, _statistics, _computedOutput) } def computedOutput: RDD[InternalRow] = this.synchronized { if (_computedOutput == null) { - _computedOutput = child.execute() + _computedOutput = executedChild.execute() } _computedOutput } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala index 3c45ee0b62dd3..8c2cca9fbef64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala @@ -22,25 +22,25 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics private[sql] case class CommonSubqueryExec( attributes: Seq[Attribute], @transient subquery: CommonSubquery) - extends LeafExecNode { + extends UnaryExecNode { - override protected def innerChildren: Seq[QueryPlan[_]] = Seq(subquery) ++ super.innerChildren + override val child: SparkPlan = subquery.executedChild override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def output: Seq[Attribute] = attributes - override def outputPartitioning: Partitioning = subquery.child.outputPartitioning + override def outputPartitioning: Partitioning = subquery.executedChild.outputPartitioning - override def outputOrdering: Seq[SortOrder] = subquery.child.outputOrdering + override def outputOrdering: Seq[SortOrder] = subquery.executedChild.outputOrdering protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala index 1625378dcee43..f0d11ffbf2a84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala @@ -50,8 +50,9 @@ private[sql] class SubqueryDedup { CommonSubqueryItem( planToDedup, CommonSubquery(planToDedup.output, - sparkSession.sessionState.executePlan(planToDedup).executedPlan) - (planToDedup.statistics)) + planToDedup) + (sparkSession.sessionState.executePlan(planToDedup).executedPlan, + planToDedup.statistics)) subqueryData += common common.subquery }.withOutput(planToDedup.output) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index bf9c8ebfa897b..2981fa09d3935 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -583,7 +583,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { val df = sql("WITH s AS (SELECT 1 FROM dedup) SELECT * FROM s s1 join s s2") val commonSubqueries = df.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child + case c: CommonSubqueryExec => c.subquery.executedChild }.distinct assert(commonSubqueries.length == 1) @@ -591,7 +591,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { "SELECT * FROM s1 JOIN (SELECT * FROM s1, s2)") val commonSubqueries2 = df2.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child + case c: CommonSubqueryExec => c.subquery.executedChild }.distinct assert(commonSubqueries2.length == 1) @@ -599,7 +599,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { "WHERE a.id = 1 AND b.id > 0") val commonSubqueries3 = df3.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child + case c: CommonSubqueryExec => c.subquery.executedChild }.distinct assert(commonSubqueries3.length == 1) } @@ -607,7 +607,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { // Using a self-join as CTE to test if de-duplicated attributes work for this. val df4 = sql("WITH j AS (SELECT * FROM (SELECT * FROM l JOIN l)) SELECT * FROM j j1, j j2") val commonSubqueries4 = df4.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child + case c: CommonSubqueryExec => c.subquery.executedChild }.distinct assert(commonSubqueries4.length == 1) @@ -619,7 +619,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { val df5 = sql("WITH cte AS (SELECT * FROM l a, l b), cte2 AS (SELECT * FROM cte j1, cte) " + "SELECT * FROM cte2 j3, cte j4") val commonSubqueries5 = df5.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child + case c: CommonSubqueryExec => c.subquery.executedChild }.distinct assert(commonSubqueries5.length == 1) } @@ -630,7 +630,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { "SELECT * FROM (SELECT * FROM cte WHERE a = 1) x JOIN (SELECT * FROM cte WHERE b = 1.0) y") val commonSubqueries = df.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child + case c: CommonSubqueryExec => c.subquery.executedChild }.distinct assert(commonSubqueries.length == 1) val pushdownFilter = commonSubqueries(0).collect { @@ -655,7 +655,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { "SELECT * FROM cte x, cte y WHERE x.b = y.b AND x.a = 1 AND x.b = 1 + 1") val commonSubqueries2 = df2.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child + case c: CommonSubqueryExec => c.subquery.executedChild }.distinct assert(commonSubqueries2.length == 1) val pushdownFilter2 = commonSubqueries2(0).collect { @@ -680,7 +680,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { "WHERE a = (SELECT max(b) FROM t1)) SELECT * FROM t2 x UNION ALL SELECT * FROM t2 y") checkAnswer(df, Array(Row(1), Row(1))) val commonSubqueries = df.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child + case c: CommonSubqueryExec => c.subquery.executedChild }.distinct assert(commonSubqueries.length == 1) } @@ -691,7 +691,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { "SELECT * FROM (SELECT a FROM cte) x JOIN (SELECT b FROM cte) y") val commonSubqueries = df.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child + case c: CommonSubqueryExec => c.subquery.executedChild }.distinct assert(commonSubqueries.length == 1) From 38c57e854f2f355de9a4a8a01fd2220ae86418eb Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 30 Aug 2016 07:16:05 +0800 Subject: [PATCH 18/28] Add executed plan into otherCopyArgs. --- .../apache/spark/sql/execution/subquery/CommonSubquery.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala index b6d6eff254686..d0e4eb3bb0f01 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala @@ -51,5 +51,5 @@ private[sql] case class CommonSubquery( } override protected def otherCopyArgs: Seq[AnyRef] = - Seq(_statistics, _computedOutput) + Seq(executedChild, _statistics, _computedOutput) } From 49eefb06e67a54fe14b4ea94e905a35a05e63f47 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 30 Aug 2016 14:24:30 +0800 Subject: [PATCH 19/28] Fix the issue. --- .../execution/subquery/CommonSubquery.scala | 17 +++++++++++------ .../subquery/CommonSubqueryExec.scala | 12 ++++++------ .../sql/execution/subquery/SubqueryDedup.scala | 4 ++-- .../org/apache/spark/sql/SubquerySuite.scala | 18 +++++++++--------- 4 files changed, 28 insertions(+), 23 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala index d0e4eb3bb0f01..11d5af08ea438 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala @@ -24,14 +24,19 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.util.Utils private[sql] case class CommonSubquery( output: Seq[Attribute], - @transient child: LogicalPlan)( - @transient val executedChild: SparkPlan, + @transient child: SparkPlan)( + @transient val logicalChild: LogicalPlan, private[sql] val _statistics: Statistics, @transient private[sql] var _computedOutput: RDD[InternalRow] = null) - extends logical.UnaryNode { + extends logical.LeafNode { + + override def argString: String = Utils.truncatedString(output, "[", ", ", "]") + + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(logicalChild) override def producedAttributes: AttributeSet = outputSet @@ -40,16 +45,16 @@ private[sql] case class CommonSubquery( lazy val numRows: Long = computedOutput.count def withOutput(newOutput: Seq[Attribute]): CommonSubquery = { - CommonSubquery(newOutput, child)(executedChild, _statistics, _computedOutput) + CommonSubquery(newOutput, child)(logicalChild, _statistics, _computedOutput) } def computedOutput: RDD[InternalRow] = this.synchronized { if (_computedOutput == null) { - _computedOutput = executedChild.execute() + _computedOutput = child.execute() } _computedOutput } override protected def otherCopyArgs: Seq[AnyRef] = - Seq(executedChild, _statistics, _computedOutput) + Seq(logicalChild, _statistics, _computedOutput) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala index 8c2cca9fbef64..2fcfe108469d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala @@ -22,25 +22,25 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.{LeafExecNode, UnaryExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics - +import org.apache.spark.util.Utils private[sql] case class CommonSubqueryExec( attributes: Seq[Attribute], @transient subquery: CommonSubquery) - extends UnaryExecNode { + extends LeafExecNode { - override val child: SparkPlan = subquery.executedChild + override def argString: String = Utils.truncatedString(attributes, "[", ", ", "]") override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def output: Seq[Attribute] = attributes - override def outputPartitioning: Partitioning = subquery.executedChild.outputPartitioning + override def outputPartitioning: Partitioning = subquery.child.outputPartitioning - override def outputOrdering: Seq[SortOrder] = subquery.executedChild.outputOrdering + override def outputOrdering: Seq[SortOrder] = subquery.child.outputOrdering protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala index f0d11ffbf2a84..e24099a8b2033 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala @@ -50,8 +50,8 @@ private[sql] class SubqueryDedup { CommonSubqueryItem( planToDedup, CommonSubquery(planToDedup.output, - planToDedup) - (sparkSession.sessionState.executePlan(planToDedup).executedPlan, + sparkSession.sessionState.executePlan(planToDedup).executedPlan) + (planToDedup, planToDedup.statistics)) subqueryData += common common.subquery diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 2981fa09d3935..bf9c8ebfa897b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -583,7 +583,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { val df = sql("WITH s AS (SELECT 1 FROM dedup) SELECT * FROM s s1 join s s2") val commonSubqueries = df.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.executedChild + case c: CommonSubqueryExec => c.subquery.child }.distinct assert(commonSubqueries.length == 1) @@ -591,7 +591,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { "SELECT * FROM s1 JOIN (SELECT * FROM s1, s2)") val commonSubqueries2 = df2.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.executedChild + case c: CommonSubqueryExec => c.subquery.child }.distinct assert(commonSubqueries2.length == 1) @@ -599,7 +599,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { "WHERE a.id = 1 AND b.id > 0") val commonSubqueries3 = df3.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.executedChild + case c: CommonSubqueryExec => c.subquery.child }.distinct assert(commonSubqueries3.length == 1) } @@ -607,7 +607,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { // Using a self-join as CTE to test if de-duplicated attributes work for this. val df4 = sql("WITH j AS (SELECT * FROM (SELECT * FROM l JOIN l)) SELECT * FROM j j1, j j2") val commonSubqueries4 = df4.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.executedChild + case c: CommonSubqueryExec => c.subquery.child }.distinct assert(commonSubqueries4.length == 1) @@ -619,7 +619,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { val df5 = sql("WITH cte AS (SELECT * FROM l a, l b), cte2 AS (SELECT * FROM cte j1, cte) " + "SELECT * FROM cte2 j3, cte j4") val commonSubqueries5 = df5.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.executedChild + case c: CommonSubqueryExec => c.subquery.child }.distinct assert(commonSubqueries5.length == 1) } @@ -630,7 +630,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { "SELECT * FROM (SELECT * FROM cte WHERE a = 1) x JOIN (SELECT * FROM cte WHERE b = 1.0) y") val commonSubqueries = df.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.executedChild + case c: CommonSubqueryExec => c.subquery.child }.distinct assert(commonSubqueries.length == 1) val pushdownFilter = commonSubqueries(0).collect { @@ -655,7 +655,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { "SELECT * FROM cte x, cte y WHERE x.b = y.b AND x.a = 1 AND x.b = 1 + 1") val commonSubqueries2 = df2.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.executedChild + case c: CommonSubqueryExec => c.subquery.child }.distinct assert(commonSubqueries2.length == 1) val pushdownFilter2 = commonSubqueries2(0).collect { @@ -680,7 +680,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { "WHERE a = (SELECT max(b) FROM t1)) SELECT * FROM t2 x UNION ALL SELECT * FROM t2 y") checkAnswer(df, Array(Row(1), Row(1))) val commonSubqueries = df.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.executedChild + case c: CommonSubqueryExec => c.subquery.child }.distinct assert(commonSubqueries.length == 1) } @@ -691,7 +691,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { "SELECT * FROM (SELECT a FROM cte) x JOIN (SELECT b FROM cte) y") val commonSubqueries = df.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.executedChild + case c: CommonSubqueryExec => c.subquery.child }.distinct assert(commonSubqueries.length == 1) From 79c45ac9ed1e99e6729f9a66d4d0872c5a529e35 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 30 Aug 2016 16:00:37 +0800 Subject: [PATCH 20/28] Continue fixing query plan string. --- .../spark/sql/execution/subquery/CommonSubqueryExec.scala | 2 ++ .../apache/spark/sql/execution/subquery/SubqueryDedup.scala | 5 +++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala index 2fcfe108469d2..e61396fd626be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubqueryExec.scala @@ -31,6 +31,8 @@ private[sql] case class CommonSubqueryExec( @transient subquery: CommonSubquery) extends LeafExecNode { + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(subquery.child) + override def argString: String = Utils.truncatedString(attributes, "[", ", ", "]") override lazy val metrics = Map( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala index e24099a8b2033..0870ad7c2ff03 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala @@ -45,13 +45,14 @@ private[sql] class SubqueryDedup { private[sql] def createCommonSubquery( sparkSession: SparkSession, planToDedup: LogicalPlan): CommonSubquery = { + val execution = sparkSession.sessionState.executePlan(planToDedup) lookupCommonSubquery(planToDedup).map(_.subquery).getOrElse { val common = CommonSubqueryItem( planToDedup, CommonSubquery(planToDedup.output, - sparkSession.sessionState.executePlan(planToDedup).executedPlan) - (planToDedup, + execution.executedPlan) + (execution.optimizedPlan, planToDedup.statistics)) subqueryData += common common.subquery From 5729bb2f67dfa2435d2371039c349d99050cfef8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 31 Aug 2016 20:23:40 +0800 Subject: [PATCH 21/28] Improve filter pushdown. --- .../sql/catalyst/optimizer/Optimizer.scala | 40 +++++++++++++--- .../org/apache/spark/sql/SubquerySuite.scala | 48 +++++++++++++++++-- 2 files changed, 76 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 61c1741c4ceb0..b893d3cc32410 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1137,13 +1137,39 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) plan: LogicalPlan, keyPlan: LogicalPlan, subqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = { - val firstCondExprs = splitConjunctivePredicates(subqueries(0).asInstanceOf[Filter].condition) - val firstPushdownCondition: Expression = (subqueries(0).asInstanceOf[Filter].condition +: - subqueries.tail.map { - case Filter(otherCond, child) => - val rewrites = buildRewrites(child, subqueries(0).asInstanceOf[Filter].child) - pushToOtherPlan(otherCond, rewrites) - }).reduce(Or) + val orConds = new ArrayBuffer[Expression]() + var pushdownConds = splitConjunctivePredicates(subqueries(0).asInstanceOf[Filter].condition) + subqueries.tail.foreach { + case Filter(otherCond, child) => + val rewrites = buildRewrites(child, subqueries(0).asInstanceOf[Filter].child) + // We can't simply push down all conditions from other Filter by concatenating them with + // [[Or]]. Because if any conditions contains a sub-condition which can't be pushed down + // through intermediate operators, it makes all concatenated conditions not pushed doen. + // E.g., first condition is [a && b] and second condition is [c]. If b can't be pushed + // down, the final condition [[a && b] || c] can't be pushed down too. + // So we extract the sub-conditions which share the same references and only push down + // them. + splitConjunctivePredicates(otherCond).foreach { cond => + val rewritten = pushToOtherPlan(cond, rewrites) + if (!pushdownConds.exists(_.semanticEquals(rewritten))) { + var found = false + pushdownConds = pushdownConds.map { pushdown => + if (!found && rewritten.references.subsetOf(pushdown.references)) { + found = true + Or(pushdown, rewritten) + } else { + pushdown + } + } + if (!found) orConds += rewritten + } + } + } + val firstPushdownCondition: Expression = if (orConds.isEmpty) { + pushdownConds.reduce(And) + } else { + Or(pushdownConds.reduce(And), orConds.reduce(Or)) + } plan transformDown { case f @ Filter(cond, s @ SubqueryAlias(alias, subquery, v, true)) if s.sameResult(keyPlan) => val pushdownCond: Expression = subqueries.foldLeft(firstPushdownCondition) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index bf9c8ebfa897b..42618d812affa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, GreaterThan, Literal} import org.apache.spark.sql.execution.{FilterExec, LocalTableScanExec} import org.apache.spark.sql.execution.subquery.CommonSubqueryExec import org.apache.spark.sql.internal.SQLConf @@ -50,6 +50,22 @@ class SubquerySuite extends QueryTest with SharedSQLContext { row(null, 5.0), row(6, null)).toDF("c", "d") + lazy val tab1 = Seq( + row(1, 2), + row(2, 3), + row(5, 6), + row(6, 7), + row(10, 11), + row(11, 12)).toDF("c1", "c2") + + lazy val tab2 = Seq( + row(1, 2), + row(2, 3), + row(5, 6), + row(6, 7), + row(10, 11), + row(11, 12)).toDF("c1", "c2") + lazy val t = r.filter($"c".isNotNull && $"d".isNotNull) protected override def beforeAll(): Unit = { @@ -57,6 +73,8 @@ class SubquerySuite extends QueryTest with SharedSQLContext { l.createOrReplaceTempView("l") r.createOrReplaceTempView("r") t.createOrReplaceTempView("t") + tab1.createOrReplaceTempView("tab1") + tab2.createOrReplaceTempView("tab2") } test("rdd deserialization does not crash [SPARK-15791]") { @@ -628,7 +646,6 @@ class SubquerySuite extends QueryTest with SharedSQLContext { withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { val df = sql("WITH cte AS (SELECT a.a AS a, a.b AS b, b.a AS c, b.b AS d FROM l a, l b) " + "SELECT * FROM (SELECT * FROM cte WHERE a = 1) x JOIN (SELECT * FROM cte WHERE b = 1.0) y") - val commonSubqueries = df.queryExecution.sparkPlan.collect { case c: CommonSubqueryExec => c.subquery.child }.distinct @@ -650,10 +667,9 @@ class SubquerySuite extends QueryTest with SharedSQLContext { // There are two Filters: // 1. x.a = 1 && x.b = 2.0 // 2. x.b = 2.0 - // The pushdown Filter is ((x.a = 1 && x.b = 2.0) || (x.b = 2.0)) => (x.b = 2.0). + // The conditions (x.a = 1 && x.b = 2.0) should be pushed down. val df2 = sql("WITH cte AS (SELECT a.a AS a, a.b AS b, b.a AS c, b.b AS d FROM l a, l b) " + "SELECT * FROM cte x, cte y WHERE x.b = y.b AND x.a = 1 AND x.b = 1 + 1") - val commonSubqueries2 = df2.queryExecution.sparkPlan.collect { case c: CommonSubqueryExec => c.subquery.child }.distinct @@ -665,12 +681,34 @@ class SubquerySuite extends QueryTest with SharedSQLContext { val intConditions2 = pushdownFilter2(0).asInstanceOf[FilterExec].condition.collect { case EqualTo(a: AttributeReference, Literal(i, IntegerType)) => (a.name, i) } - assert(intConditions2.length == 0) + assert(intConditions2.length == 1 && + intConditions2(0)._1 == "a" && intConditions2(0)._2 == 1) val doubleConditions2 = pushdownFilter2(0).asInstanceOf[FilterExec].condition.collect { case EqualTo(a: AttributeReference, Literal(d, DoubleType)) => (a.name, d) } assert(doubleConditions2.length == 1 && doubleConditions2(0)._1 == "b" && doubleConditions2(0)._2 == 2.0) + + val df3 = sql("with cte as (select c1, case mycount when 0 then null else 1 end cov " + + "from (select tab1.c1, count(tab1.c2) as mycount from tab1, tab2 " + + "where tab1.c1 = tab2.c1 group by tab1.c1) foo " + + "where case mycount when 0 then null else 1 end > 1.0) " + + "select * from cte a, cte b where a.c1 > 5 and b.c1 > 10 and a.cov > 2") + val commonSubqueries3 = df3.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries3.length == 1) + val pushdownFilter3 = commonSubqueries3(0).collect { + case f: FilterExec => f + } + // Besides the original Filter, two Filters are pushed down: + // One is tab1.c1 > 5 or tab1.c1 > 10, another one is tab2.c1 > 5 or tab2.c1 > 10. + assert(pushdownFilter3.length == 3) + val intConditions3 = pushdownFilter3(1).asInstanceOf[FilterExec].condition.collect { + case GreaterThan(a: AttributeReference, Literal(i, IntegerType)) => (a.name, i) + } + assert(intConditions3.length == 2) + assert(intConditions3(0)._2 == 5 && intConditions3(1)._2 == 10) } } From e9b09527ca98b3f99b43be3a028f04a207422389 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 1 Sep 2016 12:59:04 +0800 Subject: [PATCH 22/28] Fix filter pushdown again. --- .../sql/catalyst/optimizer/Optimizer.scala | 27 +++++-------------- .../org/apache/spark/sql/SubquerySuite.scala | 10 +++---- 2 files changed, 12 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index b893d3cc32410..434374dd5d17c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -90,6 +90,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) CombineFilters, CombineLimits, CombineUnions, + // Pushdown Filters again after combination + PushDownPredicate, // Constant folding and strength reduction NullPropagation, FoldablePropagation, @@ -1137,8 +1139,8 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) plan: LogicalPlan, keyPlan: LogicalPlan, subqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = { - val orConds = new ArrayBuffer[Expression]() - var pushdownConds = splitConjunctivePredicates(subqueries(0).asInstanceOf[Filter].condition) + val pushdownConds = new ArrayBuffer[Expression]() + var firstConds = splitConjunctivePredicates(subqueries(0).asInstanceOf[Filter].condition) subqueries.tail.foreach { case Filter(otherCond, child) => val rewrites = buildRewrites(child, subqueries(0).asInstanceOf[Filter].child) @@ -1147,29 +1149,14 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) // through intermediate operators, it makes all concatenated conditions not pushed doen. // E.g., first condition is [a && b] and second condition is [c]. If b can't be pushed // down, the final condition [[a && b] || c] can't be pushed down too. - // So we extract the sub-conditions which share the same references and only push down - // them. splitConjunctivePredicates(otherCond).foreach { cond => val rewritten = pushToOtherPlan(cond, rewrites) - if (!pushdownConds.exists(_.semanticEquals(rewritten))) { - var found = false - pushdownConds = pushdownConds.map { pushdown => - if (!found && rewritten.references.subsetOf(pushdown.references)) { - found = true - Or(pushdown, rewritten) - } else { - pushdown - } - } - if (!found) orConds += rewritten + pushdownConds ++= firstConds.map { pushdown => + Or(pushdown, rewritten) } } } - val firstPushdownCondition: Expression = if (orConds.isEmpty) { - pushdownConds.reduce(And) - } else { - Or(pushdownConds.reduce(And), orConds.reduce(Or)) - } + val firstPushdownCondition: Expression = pushdownConds.reduce(And) plan transformDown { case f @ Filter(cond, s @ SubqueryAlias(alias, subquery, v, true)) if s.sameResult(keyPlan) => val pushdownCond: Expression = subqueries.foldLeft(firstPushdownCondition) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 42618d812affa..0942d0d09092b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -656,11 +656,11 @@ class SubquerySuite extends QueryTest with SharedSQLContext { assert(pushdownFilter.length == 1) val intConditions = pushdownFilter(0).asInstanceOf[FilterExec].condition.collect { case EqualTo(a: AttributeReference, Literal(i, IntegerType)) => (a.name, i) - } + }.distinct assert(intConditions.length == 1 && intConditions(0)._1 == "a" && intConditions(0)._2 == 1) val doubleConditions = pushdownFilter(0).asInstanceOf[FilterExec].condition.collect { case EqualTo(a: AttributeReference, Literal(d, DoubleType)) => (a.name, d) - } + }.distinct assert(doubleConditions.length == 1 && doubleConditions(0)._1 == "b" && doubleConditions(0)._2 == 1.0) @@ -680,12 +680,12 @@ class SubquerySuite extends QueryTest with SharedSQLContext { assert(pushdownFilter2.length == 1) val intConditions2 = pushdownFilter2(0).asInstanceOf[FilterExec].condition.collect { case EqualTo(a: AttributeReference, Literal(i, IntegerType)) => (a.name, i) - } + }.distinct assert(intConditions2.length == 1 && intConditions2(0)._1 == "a" && intConditions2(0)._2 == 1) val doubleConditions2 = pushdownFilter2(0).asInstanceOf[FilterExec].condition.collect { case EqualTo(a: AttributeReference, Literal(d, DoubleType)) => (a.name, d) - } + }.distinct assert(doubleConditions2.length == 1 && doubleConditions2(0)._1 == "b" && doubleConditions2(0)._2 == 2.0) @@ -706,7 +706,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { assert(pushdownFilter3.length == 3) val intConditions3 = pushdownFilter3(1).asInstanceOf[FilterExec].condition.collect { case GreaterThan(a: AttributeReference, Literal(i, IntegerType)) => (a.name, i) - } + }.distinct assert(intConditions3.length == 2) assert(intConditions3(0)._2 == 5 && intConditions3(1)._2 == 10) } From e1d050fac9a9071a677101e5e023d30ff9dcc7ee Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 3 Sep 2016 17:11:03 +0800 Subject: [PATCH 23/28] Fix subquery execution. --- .../execution/subquery/CommonSubquery.scala | 24 +++--- .../execution/subquery/SubqueryDedup.scala | 3 +- .../org/apache/spark/sql/SubquerySuite.scala | 74 ++++++++++--------- .../subquery/SubqueryDedupSuite.scala | 47 ++++++------ 4 files changed, 80 insertions(+), 68 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala index 11d5af08ea438..20d128c50b2cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala @@ -26,12 +26,23 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.util.Utils +private[sql] case class SubqueryExecHelper(executedPlan: SparkPlan) { + private var _computedOutput: RDD[InternalRow] = null + + def computeOrGetResult(): RDD[InternalRow] = this.synchronized { + if (_computedOutput == null) { + _computedOutput = executedPlan.execute() + } + _computedOutput + } +} + private[sql] case class CommonSubquery( output: Seq[Attribute], @transient child: SparkPlan)( @transient val logicalChild: LogicalPlan, private[sql] val _statistics: Statistics, - @transient private[sql] var _computedOutput: RDD[InternalRow] = null) + @transient private val execHelper: SubqueryExecHelper) extends logical.LeafNode { override def argString: String = Utils.truncatedString(output, "[", ", ", "]") @@ -45,16 +56,11 @@ private[sql] case class CommonSubquery( lazy val numRows: Long = computedOutput.count def withOutput(newOutput: Seq[Attribute]): CommonSubquery = { - CommonSubquery(newOutput, child)(logicalChild, _statistics, _computedOutput) + CommonSubquery(newOutput, child)(logicalChild, _statistics, execHelper) } - def computedOutput: RDD[InternalRow] = this.synchronized { - if (_computedOutput == null) { - _computedOutput = child.execute() - } - _computedOutput - } + def computedOutput: RDD[InternalRow] = execHelper.computeOrGetResult() override protected def otherCopyArgs: Seq[AnyRef] = - Seq(logicalChild, _statistics, _computedOutput) + Seq(logicalChild, _statistics, execHelper) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala index 0870ad7c2ff03..bf3f358a58b62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/SubqueryDedup.scala @@ -53,7 +53,8 @@ private[sql] class SubqueryDedup { CommonSubquery(planToDedup.output, execution.executedPlan) (execution.optimizedPlan, - planToDedup.statistics)) + planToDedup.statistics, + SubqueryExecHelper(execution.executedPlan))) subqueryData += common common.subquery }.withOutput(planToDedup.output) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 0942d0d09092b..f3265bd1125c8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -596,50 +596,52 @@ class SubquerySuite extends QueryTest with SharedSQLContext { } test("Dedup subqueries") { - withTempView("dedup") { - spark.range(10).createOrReplaceTempView("dedup") - val df = sql("WITH s AS (SELECT 1 FROM dedup) SELECT * FROM s s1 join s s2") + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + withTempView("dedup") { + spark.range(10).createOrReplaceTempView("dedup") + val df = sql("WITH s AS (SELECT 1 FROM dedup) SELECT * FROM s s1 join s s2") + + val commonSubqueries = df.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries.length == 1) + + val df2 = sql("WITH s1 AS (SELECT 1 FROM dedup), s2 AS (SELECT 1 FROM dedup) " + + "SELECT * FROM s1 JOIN (SELECT * FROM s1, s2)") + + val commonSubqueries2 = df2.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries2.length == 1) + + val df3 = sql("WITH t1 AS (SELECT 1 AS id FROM dedup) SELECT * FROM t1 a, t1 b " + + "WHERE a.id = 1 AND b.id > 0") + + val commonSubqueries3 = df3.queryExecution.sparkPlan.collect { + case c: CommonSubqueryExec => c.subquery.child + }.distinct + assert(commonSubqueries3.length == 1) + } - val commonSubqueries = df.queryExecution.sparkPlan.collect { + // Using a self-join as CTE to test if de-duplicated attributes work for this. + val df4 = sql("WITH j AS (SELECT * FROM (SELECT * FROM l JOIN l)) SELECT * FROM j j1, j j2") + val commonSubqueries4 = df4.queryExecution.sparkPlan.collect { case c: CommonSubqueryExec => c.subquery.child }.distinct - assert(commonSubqueries.length == 1) + assert(commonSubqueries4.length == 1) - val df2 = sql("WITH s1 AS (SELECT 1 FROM dedup), s2 AS (SELECT 1 FROM dedup) " + - "SELECT * FROM s1 JOIN (SELECT * FROM s1, s2)") + val df4WithoutCTE = sql("SELECT * FROM (SELECT * FROM (SELECT * FROM l JOIN l)) j1, " + + "(SELECT * FROM (SELECT * FROM l JOIN l)) j2") + assert(df4.collect() === df4WithoutCTE.collect()) - val commonSubqueries2 = df2.queryExecution.sparkPlan.collect { + // CTE subquery refers to previous CTE subquery. + val df5 = sql("WITH cte AS (SELECT * FROM l a, l b), cte2 AS (SELECT * FROM cte j1, cte) " + + "SELECT * FROM cte2 j3, cte j4") + val commonSubqueries5 = df5.queryExecution.sparkPlan.collect { case c: CommonSubqueryExec => c.subquery.child }.distinct - assert(commonSubqueries2.length == 1) - - val df3 = sql("WITH t1 AS (SELECT 1 AS id FROM dedup) SELECT * FROM t1 a, t1 b " + - "WHERE a.id = 1 AND b.id > 0") - - val commonSubqueries3 = df3.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child - }.distinct - assert(commonSubqueries3.length == 1) + assert(commonSubqueries5.length == 1) } - - // Using a self-join as CTE to test if de-duplicated attributes work for this. - val df4 = sql("WITH j AS (SELECT * FROM (SELECT * FROM l JOIN l)) SELECT * FROM j j1, j j2") - val commonSubqueries4 = df4.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child - }.distinct - assert(commonSubqueries4.length == 1) - - val df4WithoutCTE = sql("SELECT * FROM (SELECT * FROM (SELECT * FROM l JOIN l)) j1, " + - "(SELECT * FROM (SELECT * FROM l JOIN l)) j2") - assert(df4.collect() === df4WithoutCTE.collect()) - - // CTE subquery refers to previous CTE subquery. - val df5 = sql("WITH cte AS (SELECT * FROM l a, l b), cte2 AS (SELECT * FROM cte j1, cte) " + - "SELECT * FROM cte2 j3, cte j4") - val commonSubqueries5 = df5.queryExecution.sparkPlan.collect { - case c: CommonSubqueryExec => c.subquery.child - }.distinct - assert(commonSubqueries5.length == 1) } test("Dedup subqueries with optimization: Filter pushdown") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/subquery/SubqueryDedupSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/subquery/SubqueryDedupSuite.scala index 1477167687520..02510685f9c43 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/subquery/SubqueryDedupSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/subquery/SubqueryDedupSuite.scala @@ -19,33 +19,36 @@ package org.apache.spark.sql.execution.subquery import org.apache.spark.sql.{Dataset, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext class SubqueryDedupSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("Create common subquery") { - val subqueryDedup = new SubqueryDedup() - - spark.range(10).createOrReplaceTempView("t1") - val df = sql("SELECT 1 FROM t1") - val project = df.queryExecution.analyzed.collect { - case p: Project => p - }.head - - val commonSubqueryAlias = subqueryDedup.createCommonSubquery(spark, project) - assert(commonSubqueryAlias.output === project.output) - assert(Dataset.ofRows(spark, commonSubqueryAlias).collect() === df.collect()) - - spark.range(10).createOrReplaceTempView("t2") - - val df2 = sql("SELECT * FROM t1 join t2") - val project2 = df2.queryExecution.analyzed.collect { - case p: Project => p - }.head - - val commonSubqueryAlias2 = subqueryDedup.createCommonSubquery(spark, project2) - assert(commonSubqueryAlias2.output === project2.output) - assert(Dataset.ofRows(spark, commonSubqueryAlias2).collect() === df2.collect()) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + val subqueryDedup = new SubqueryDedup() + + spark.range(10).createOrReplaceTempView("t1") + val df = sql("SELECT 1 FROM t1") + val project = df.queryExecution.analyzed.collect { + case p: Project => p + }.head + + val commonSubqueryAlias = subqueryDedup.createCommonSubquery(spark, project) + assert(commonSubqueryAlias.output === project.output) + assert(Dataset.ofRows(spark, commonSubqueryAlias).collect() === df.collect()) + + spark.range(10).createOrReplaceTempView("t2") + + val df2 = sql("SELECT * FROM t1 join t2") + val project2 = df2.queryExecution.analyzed.collect { + case p: Project => p + }.head + + val commonSubqueryAlias2 = subqueryDedup.createCommonSubquery(spark, project2) + assert(commonSubqueryAlias2.output === project2.output) + assert(Dataset.ofRows(spark, commonSubqueryAlias2).collect() === df2.collect()) + } } } From 6d79bebacd8e9f0672c713c1f954502dbda3f992 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 5 Sep 2016 12:58:46 +0800 Subject: [PATCH 24/28] Fix the cnf. --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index b7b2f3b8dc07b..7e5c7f9334d93 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1182,8 +1182,7 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) plan: LogicalPlan, keyPlan: LogicalPlan, subqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = { - val pushdownConds = new ArrayBuffer[Expression]() - var firstConds = splitConjunctivePredicates(subqueries(0).asInstanceOf[Filter].condition) + var pushdownConds = splitConjunctivePredicates(subqueries(0).asInstanceOf[Filter].condition) subqueries.tail.foreach { case Filter(otherCond, child) => val rewrites = buildRewrites(child, subqueries(0).asInstanceOf[Filter].child) @@ -1192,17 +1191,19 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) // through intermediate operators, it makes all concatenated conditions not pushed doen. // E.g., first condition is [a && b] and second condition is [c]. If b can't be pushed // down, the final condition [[a && b] || c] can't be pushed down too. + val pushdowns = new ArrayBuffer[Expression]() splitConjunctivePredicates(otherCond).foreach { cond => val rewritten = pushToOtherPlan(cond, rewrites) - pushdownConds ++= firstConds.map { pushdown => + pushdowns ++= pushdownConds.map { pushdown => Or(pushdown, rewritten) } } + pushdownConds = pushdowns.toSeq } - val firstPushdownCondition: Expression = pushdownConds.reduce(And) + val finalPushdownCondition: Expression = pushdownConds.reduce(And) plan transformDown { case f @ Filter(cond, s @ SubqueryAlias(alias, subquery, v, true)) if s.sameResult(keyPlan) => - val pushdownCond: Expression = subqueries.foldLeft(firstPushdownCondition) { + val pushdownCond: Expression = subqueries.foldLeft(finalPushdownCondition) { case (currentCond, sub) => val rewrites = buildRewrites(sub.asInstanceOf[Filter].child, subquery) pushToOtherPlan(currentCond, rewrites) From bc70354fefeb2ff2cac57d869b6d342230859fd3 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 9 Sep 2016 11:38:29 +0800 Subject: [PATCH 25/28] Fix predicate explosion. --- .../sql/catalyst/optimizer/Optimizer.scala | 22 ++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 7e5c7f9334d93..2d3eca7281e8a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1169,7 +1169,7 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) * Rewrites an expression so that it can be pushed to another LogicalPlan. */ private def pushToOtherPlan[A <: Expression](e: A, rewrites: AttributeMap[Attribute]) = { - val result = e transform { + val result = e transformUp { case a: Attribute => rewrites.get(a).getOrElse(a) } @@ -1194,8 +1194,24 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) val pushdowns = new ArrayBuffer[Expression]() splitConjunctivePredicates(otherCond).foreach { cond => val rewritten = pushToOtherPlan(cond, rewrites) - pushdowns ++= pushdownConds.map { pushdown => - Or(pushdown, rewritten) + pushdownConds.flatMap { pushdown => + val subConds = splitDisjunctivePredicates(pushdown) + val orCond = Or(pushdown, rewritten) + // To avoid exponential explosion of predicates, we skip [[IsNotNull]] and predicates + // which semantically equal to existing predicates. + if (rewritten.isInstanceOf[IsNotNull] + || pushdown.isInstanceOf[IsNotNull] + || subConds.exists(rewritten.semanticEquals(_)) + || pushdowns.exists(orCond.semanticEquals(_)) + || pushdownConds.exists(orCond.semanticEquals(_))) { + None + } else { + Some(orCond) + } + }.map { cond => + if (!pushdowns.exists(cond.semanticEquals(_))) { + pushdowns += cond + } } } pushdownConds = pushdowns.toSeq From 23e2dc865eef690eb273cc69888ca577eaa603a2 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 9 Sep 2016 14:48:18 +0800 Subject: [PATCH 26/28] Deal with no pushdown predicate. --- .../sql/catalyst/optimizer/Optimizer.scala | 39 +++++++++++-------- 1 file changed, 22 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 2d3eca7281e8a..a2fc128bc7046 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1216,25 +1216,30 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) } pushdownConds = pushdowns.toSeq } - val finalPushdownCondition: Expression = pushdownConds.reduce(And) - plan transformDown { - case f @ Filter(cond, s @ SubqueryAlias(alias, subquery, v, true)) if s.sameResult(keyPlan) => - val pushdownCond: Expression = subqueries.foldLeft(finalPushdownCondition) { - case (currentCond, sub) => - val rewrites = buildRewrites(sub.asInstanceOf[Filter].child, subquery) - pushToOtherPlan(currentCond, rewrites) - } + // No pushdown for common subqueries. + if (pushdownConds.isEmpty) { + plan + } else { + val finalPushdownCondition: Expression = pushdownConds.reduce(And) + plan transformDown { + case f @ Filter(cond, s @ SubqueryAlias(a, subquery, v, true)) if s.sameResult(keyPlan) => + val pushdownCond: Expression = subqueries.foldLeft(finalPushdownCondition) { + case (currentCond, sub) => + val rewrites = buildRewrites(sub.asInstanceOf[Filter].child, subquery) + pushToOtherPlan(currentCond, rewrites) + } - val newSubquery = Filter(pushdownCond, subquery) - val optimized = optimizer.execute(newSubquery) + val newSubquery = Filter(pushdownCond, subquery) + val optimized = optimizer.execute(newSubquery) - // Check if any optimization is performed. - if (optimized.sameResult(newSubquery)) { - // No optimization happens. Let's keep original subquery. - f - } else { - Filter(cond, SubqueryAlias(alias, newSubquery, v, true)) - } + // Check if any optimization is performed. + if (optimized.sameResult(newSubquery)) { + // No optimization happens. Let's keep original subquery. + f + } else { + Filter(cond, SubqueryAlias(a, newSubquery, v, true)) + } + } } } From f153c124ed412d6624cd2811637b9b71541e7e3b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 26 Dec 2016 08:56:48 +0000 Subject: [PATCH 27/28] Cache the rdd of common subquery. --- .../spark/sql/catalyst/expressions/UnsafeRow.java | 11 +++++++++++ .../sql/execution/subquery/CommonSubquery.scala | 13 ++++++++++++- 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index d205547698c5b..a2d4e3a38905c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -172,6 +172,17 @@ public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { this.sizeInBytes = sizeInBytes; } + /** + * Update this UnsafeRow to point to different UnsafeRow. + * + * @param other the UnsafeRow to point to + */ + public void pointTo(UnsafeRow other) { + this.baseObject = other.baseObject; + this.baseOffset = other.baseOffset; + this.sizeInBytes = other.sizeInBytes; + } + /** * Update this UnsafeRow to point to the underlying byte array. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala index 20d128c50b2cd..879f18ef37e6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils private[sql] case class SubqueryExecHelper(executedPlan: SparkPlan) { @@ -31,7 +32,17 @@ private[sql] case class SubqueryExecHelper(executedPlan: SparkPlan) { def computeOrGetResult(): RDD[InternalRow] = this.synchronized { if (_computedOutput == null) { - _computedOutput = executedPlan.execute() + _computedOutput = executedPlan.execute().mapPartitionsInternal { iter => + iter.map { row => + // A hack to create new UnsafeRow so each row can be serialized individually. + val copyRow = new UnsafeRow(row.numFields) + copyRow.pointTo(row.asInstanceOf[UnsafeRow]) + copyRow.asInstanceOf[InternalRow] + } + // Use serialized cache because under this mode the rows are written to serialized format + // when iterating. If we use deseralized mode, the rows will be quoted and cached in batch. + // Because the rows are shared a common byte array, the bytes will be overritten. + }.persist(StorageLevel.MEMORY_ONLY_SER) } _computedOutput } From aeba1c31f72508c6a93d4f056a8333f4f01e6f80 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 28 Dec 2016 07:12:01 +0000 Subject: [PATCH 28/28] Cleaning some codes. --- .../sql/catalyst/optimizer/Optimizer.scala | 28 +++++++------------ .../execution/subquery/CommonSubquery.scala | 4 +-- 2 files changed, 12 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 68d23c766b8dd..4a807df492dca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -52,7 +52,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) // However, because we also use the analyzer to canonicalized queries (for view definition), // we do not eliminate subqueries or compute current time in the analyzer. Batch("Finish Analysis", Once, - EliminateOneTimeSubqueryAliases, + EliminateNonDuplicatedSubqueryAliases, ReplaceExpressions, ComputeCurrentTime, GetCurrentDatabase(sessionCatalog), @@ -1345,46 +1345,38 @@ case class OptimizeCommonSubqueries(optimizer: Optimizer) } /** - * Removes the [[SubqueryAlias]] operators which are used only once from the plan. + * Removes the [[SubqueryAlias]] operators which are not duplicated in the query plan. */ -object EliminateOneTimeSubqueryAliases extends Rule[LogicalPlan] { +object EliminateNonDuplicatedSubqueryAliases extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { val subqueries = ArrayBuffer[LogicalPlan]() val duplicateSubqueries = ArrayBuffer[LogicalPlan]() - val noRecursiveSubqueryPlan = plan.transformDown { - // Eliminate the recursive subqueries which have the same output. + // Eliminates the recursive subqueries which have the same output. + val cleanedPlan = plan.transformDown { case s @ SubqueryAlias(_, child, _, _) if child.find(p => p.isInstanceOf[SubqueryAlias] && p.sameResult(s)).isDefined => child } - noRecursiveSubqueryPlan.foreach { - // Collects the subqueries that are used more than once in the query. - // Skip the SubqueryAlias on table scan. + // Collects duplicated subqueries but ignores the SubqueryAlias of table scan. + cleanedPlan.foreach { case SubqueryAlias(_, child, _, _) if !child.isInstanceOf[MultiInstanceRelation] => if (subqueries.indexWhere(s => s.sameResult(child)) >= 0) { - // If the plan with same results can be found. duplicateSubqueries += child } else { - // If it can't be found, add it into the list. subqueries += child } case _ => } - // Set the `commonSubquery` of remainning SubqueryAlias as `true`. - noRecursiveSubqueryPlan.transformDown { - // Eliminates the subqueries that are used only once in the query. + // Eliminates non-duplicated subqueries. + cleanedPlan.transformDown { case SubqueryAlias(alias, child, v, _) => if (duplicateSubqueries.indexWhere(s => s.sameResult(child)) < 0) { child } else { - // Strip all wrapped subqueries. - val newChild = child.transformDown { - case s: SubqueryAlias => s.child - } - SubqueryAlias(alias, newChild, v, commonSubquery = true) + SubqueryAlias(alias, child, v, commonSubquery = true) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala index 879f18ef37e6f..05cba8bb4578a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery/CommonSubquery.scala @@ -40,8 +40,8 @@ private[sql] case class SubqueryExecHelper(executedPlan: SparkPlan) { copyRow.asInstanceOf[InternalRow] } // Use serialized cache because under this mode the rows are written to serialized format - // when iterating. If we use deseralized mode, the rows will be quoted and cached in batch. - // Because the rows are shared a common byte array, the bytes will be overritten. + // when iterating. If we use deseralized mode, the rows will be queued and cached in batch. + // Because the rows are shared a common byte array, the bytes will be overritten in this mode. }.persist(StorageLevel.MEMORY_ONLY_SER) } _computedOutput