From 01e3af2ef2f681e1a5841937bd272c048532ed74 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Tue, 31 May 2016 19:53:58 -0700 Subject: [PATCH 1/6] SPARK-15692 --- .../apache/spark/sql/catalyst/plans/QueryPlan.scala | 2 +- .../apache/spark/sql/catalyst/trees/TreeNode.scala | 11 ++++++++--- .../execution/columnar/InMemoryTableScanExec.scala | 5 +++++ .../apache/spark/sql/execution/command/cache.scala | 5 +++++ .../apache/spark/sql/execution/command/commands.scala | 5 ++++- .../execution/command/createDataSourceTables.scala | 3 +++ .../apache/spark/sql/execution/command/views.scala | 3 +++ .../datasources/InsertIntoDataSourceCommand.scala | 3 +++ 8 files changed, 32 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index d4447ca32d5a0..81ee596361c3e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -264,7 +264,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT expressions.flatMap(_.collect {case e: SubqueryExpression => e.plan.asInstanceOf[PlanType]}) } - override def innerChildren: Seq[PlanType] = subqueries + override protected def innerChildren: Seq[TreeNode[_]] = subqueries /** * Canonicalized copy of this query plan. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index d87e6c76ed734..03a12aa2c8574 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -424,9 +424,12 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { */ protected def stringArgs: Iterator[Any] = productIterator + private def allChildren: Set[TreeNode[_]] = containsChild ++ innerChildren.toSet[TreeNode[_]] + /** Returns a string representing the arguments to this node, minus any children */ def argString: String = productIterator.flatMap { - case tn: TreeNode[_] if containsChild(tn) => Nil + case tn: TreeNode[_] if allChildren.contains(tn) => Nil + case Some(tn: TreeNode[_]) if allChildren.contains(tn) => Nil case tn: TreeNode[_] => s"${tn.simpleString}" :: Nil case seq: Seq[BaseType] if seq.toSet.subsetOf(children.toSet) => Nil case seq: Seq[_] => seq.mkString("[", ",", "]") :: Nil @@ -467,9 +470,11 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } /** - * All the nodes that are parts of this node, this is used by subquries. + * All the nodes that are private children of current node. + * + * For example, sub-queries are stored in innerChildren. */ - protected def innerChildren: Seq[BaseType] = Nil + protected def innerChildren: Seq[TreeNode[_]] = Seq.empty /** * Appends the string represent of this node and its children to the given StringBuilder. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index ba61940b3d5a4..05374ed592c2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType @@ -70,6 +71,8 @@ private[sql] case class InMemoryRelation( private[sql] var _batchStats: ListAccumulator[InternalRow] = null) extends logical.LeafNode with MultiInstanceRelation { + override protected def innerChildren: Seq[TreeNode[_]] = Seq(child) ++ super.innerChildren + override def producedAttributes: AttributeSet = outputSet private[sql] val batchStats: ListAccumulator[InternalRow] = @@ -222,6 +225,8 @@ private[sql] case class InMemoryTableScanExec( @transient relation: InMemoryRelation) extends LeafExecNode { + override protected def innerChildren: Seq[TreeNode[_]] = Seq(relation) ++ super.innerChildren + private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index b1290a4759a25..33b748be1885f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees.TreeNode case class CacheTableCommand( @@ -28,6 +29,10 @@ case class CacheTableCommand( isLazy: Boolean) extends RunnableCommand { + override protected def innerChildren: Seq[TreeNode[_]] = { + plan.map(Seq(_)).getOrElse(Seq.empty) + } + override def run(sparkSession: SparkSession): Seq[Row] = { plan.foreach { logicalPlan => Dataset.ofRows(sparkSession, logicalPlan).createOrReplaceTempView(tableName) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 642a95a99262d..487b2e601b287 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.debug._ import org.apache.spark.sql.types._ @@ -57,6 +58,8 @@ private[sql] case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkP cmd.run(sqlContext.sparkSession).map(converter(_).asInstanceOf[InternalRow]) } + override protected def innerChildren: Seq[TreeNode[_]] = cmd :: Nil + override def output: Seq[Attribute] = cmd.output override def children: Seq[SparkPlan] = Nil @@ -69,7 +72,7 @@ private[sql] case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkP sqlContext.sparkContext.parallelize(sideEffectResult, 1) } - override def argString: String = cmd.toString + override def argString: String = "" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 4b9aab612e7c3..1b605732c277b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.sources.InsertableRelation @@ -138,6 +139,8 @@ case class CreateDataSourceTableAsSelectCommand( query: LogicalPlan) extends RunnableCommand { + override protected def innerChildren: Seq[TreeNode[_]] = Seq(query) + override def run(sparkSession: SparkSession): Seq[Row] = { // Since we are saving metadata to metastore, we need to check if metastore supports // the table name and database name we have for this query. MetaStoreUtils.validateName diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 20c02786ecc5c..d183e5b7e00c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.{SQLBuilder, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} +import org.apache.spark.sql.catalyst.trees.TreeNode /** @@ -50,6 +51,8 @@ case class CreateViewCommand( isTemporary: Boolean) extends RunnableCommand { + override protected def innerChildren: Seq[TreeNode[_]] = Seq(child) + // TODO: Note that this class can NOT canonicalize the view SQL string entirely, which is // different from Hive and may not work for some cases like create view on self join. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala index 25b901f2db8d0..03892fb3cc9c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.sources.InsertableRelation @@ -32,6 +33,8 @@ private[sql] case class InsertIntoDataSourceCommand( overwrite: Boolean) extends RunnableCommand { + override protected def innerChildren: Seq[TreeNode[_]] = Seq(query) + override def run(sparkSession: SparkSession): Seq[Row] = { val relation = logicalRelation.relation.asInstanceOf[InsertableRelation] val data = Dataset.ofRows(sparkSession, query) From c9ab285b8339814770e11c2675c8ba4d1bc730c5 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 1 Jun 2016 10:02:54 -0700 Subject: [PATCH 2/6] on wenchen's comment --- .../scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala | 2 +- .../spark/sql/execution/columnar/InMemoryTableScanExec.scala | 2 +- .../org/apache/spark/sql/execution/command/commands.scala | 3 --- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 03a12aa2c8574..0971492b4cc02 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -424,7 +424,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { */ protected def stringArgs: Iterator[Any] = productIterator - private def allChildren: Set[TreeNode[_]] = containsChild ++ innerChildren.toSet[TreeNode[_]] + private lazy val allChildren: Set[TreeNode[_]] = containsChild ++ innerChildren.toSet[TreeNode[_]] /** Returns a string representing the arguments to this node, minus any children */ def argString: String = productIterator.flatMap { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 05374ed592c2c..f089100b1b7ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -71,7 +71,7 @@ private[sql] case class InMemoryRelation( private[sql] var _batchStats: ListAccumulator[InternalRow] = null) extends logical.LeafNode with MultiInstanceRelation { - override protected def innerChildren: Seq[TreeNode[_]] = Seq(child) ++ super.innerChildren + override protected def innerChildren: Seq[TreeNode[_]] = Seq(child) override def producedAttributes: AttributeSet = outputSet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 487b2e601b287..15544801b6654 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -71,11 +71,8 @@ private[sql] case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkP protected override def doExecute(): RDD[InternalRow] = { sqlContext.sparkContext.parallelize(sideEffectResult, 1) } - - override def argString: String = "" } - /** * An explain command for users to see how a command will be executed. * From fee1afbac2c6dfda3b3a8f3d32c0a35925579d0f Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 1 Jun 2016 13:04:33 -0700 Subject: [PATCH 3/6] on wenchen's comment --- .../org/apache/spark/sql/catalyst/plans/QueryPlan.scala | 2 +- .../sql/execution/columnar/InMemoryTableScanExec.scala | 7 +++---- .../org/apache/spark/sql/execution/command/cache.scala | 5 ++--- .../org/apache/spark/sql/execution/command/commands.scala | 5 ++--- .../sql/execution/command/createDataSourceTables.scala | 4 ++-- .../org/apache/spark/sql/execution/command/views.scala | 4 ++-- .../datasources/InsertIntoDataSourceCommand.scala | 4 ++-- 7 files changed, 14 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 81ee596361c3e..6784c3ae1d7e7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -264,7 +264,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT expressions.flatMap(_.collect {case e: SubqueryExpression => e.plan.asInstanceOf[PlanType]}) } - override protected def innerChildren: Seq[TreeNode[_]] = subqueries + override protected def innerChildren: Seq[QueryPlan[_]] = subqueries /** * Canonicalized copy of this query plan. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index f089100b1b7ce..e86c40ba30378 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -28,10 +28,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical} import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType @@ -71,7 +70,7 @@ private[sql] case class InMemoryRelation( private[sql] var _batchStats: ListAccumulator[InternalRow] = null) extends logical.LeafNode with MultiInstanceRelation { - override protected def innerChildren: Seq[TreeNode[_]] = Seq(child) + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) override def producedAttributes: AttributeSet = outputSet @@ -225,7 +224,7 @@ private[sql] case class InMemoryTableScanExec( @transient relation: InMemoryRelation) extends LeafExecNode { - override protected def innerChildren: Seq[TreeNode[_]] = Seq(relation) ++ super.innerChildren + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index 33b748be1885f..728b2fce40a38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -19,9 +19,8 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.trees.TreeNode - case class CacheTableCommand( tableName: String, @@ -29,7 +28,7 @@ case class CacheTableCommand( isLazy: Boolean) extends RunnableCommand { - override protected def innerChildren: Seq[TreeNode[_]] = { + override protected def innerChildren: Seq[QueryPlan[_]] = { plan.map(Seq(_)).getOrElse(Seq.empty) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 15544801b6654..d5ef7842aa484 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -22,9 +22,8 @@ import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.debug._ import org.apache.spark.sql.types._ @@ -58,7 +57,7 @@ private[sql] case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkP cmd.run(sqlContext.sparkSession).map(converter(_).asInstanceOf[InternalRow]) } - override protected def innerChildren: Seq[TreeNode[_]] = cmd :: Nil + override protected def innerChildren: Seq[QueryPlan[_]] = cmd :: Nil override def output: Seq[Attribute] = cmd.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 1b605732c277b..b932ff269e525 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -27,8 +27,8 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.sources.InsertableRelation @@ -139,7 +139,7 @@ case class CreateDataSourceTableAsSelectCommand( query: LogicalPlan) extends RunnableCommand { - override protected def innerChildren: Seq[TreeNode[_]] = Seq(query) + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(query) override def run(sparkSession: SparkSession): Seq[Row] = { // Since we are saving metadata to metastore, we need to check if metastore supports diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index d183e5b7e00c5..b56c200e9e98b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -23,8 +23,8 @@ import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.{SQLBuilder, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} -import org.apache.spark.sql.catalyst.trees.TreeNode /** @@ -51,7 +51,7 @@ case class CreateViewCommand( isTemporary: Boolean) extends RunnableCommand { - override protected def innerChildren: Seq[TreeNode[_]] = Seq(child) + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) // TODO: Note that this class can NOT canonicalize the view SQL string entirely, which is // different from Hive and may not work for some cases like create view on self join. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala index 03892fb3cc9c0..8549ae96e2f39 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.sources.InsertableRelation @@ -33,7 +33,7 @@ private[sql] case class InsertIntoDataSourceCommand( overwrite: Boolean) extends RunnableCommand { - override protected def innerChildren: Seq[TreeNode[_]] = Seq(query) + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(query) override def run(sparkSession: SparkSession): Seq[Row] = { val relation = logicalRelation.relation.asInstanceOf[InsertableRelation] From 7ba2c1262470285956d993c39c971252d932e907 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 1 Jun 2016 14:26:38 -0700 Subject: [PATCH 4/6] fix style --- .../spark/sql/execution/columnar/InMemoryTableScanExec.scala | 3 ++- .../org/apache/spark/sql/execution/command/commands.scala | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index e86c40ba30378..7ccc9de9db233 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -28,7 +28,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical} +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.physical.Partitioning import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index d5ef7842aa484..38bb6e412f753 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -22,7 +22,8 @@ import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical} +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 import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.debug._ From dea71f795da14fb630fe5280e507969e58e63373 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 1 Jun 2016 15:37:25 -0700 Subject: [PATCH 5/6] on liancheng's comment --- .../org/apache/spark/sql/catalyst/trees/TreeNode.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 0971492b4cc02..3ebd815dce32c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -424,7 +424,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { */ protected def stringArgs: Iterator[Any] = productIterator - private lazy val allChildren: Set[TreeNode[_]] = containsChild ++ innerChildren.toSet[TreeNode[_]] + private lazy val allChildren: Set[TreeNode[_]] = (children ++ innerChildren).toSet[TreeNode[_]] /** Returns a string representing the arguments to this node, minus any children */ def argString: String = productIterator.flatMap { @@ -470,9 +470,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } /** - * All the nodes that are private children of current node. - * - * For example, sub-queries are stored in innerChildren. + * All the nodes that should be shown as a inner nested tree of this node. + * For example, this can be used to show sub-queries. */ protected def innerChildren: Seq[TreeNode[_]] = Seq.empty From 7860982db8f7bc7bb2eeaaacca6c694d789ced05 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 1 Jun 2016 16:00:13 -0700 Subject: [PATCH 6/6] on wenchen's comment --- .../scala/org/apache/spark/sql/execution/command/cache.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index 728b2fce40a38..3e5eed2efa76b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -29,7 +29,7 @@ case class CacheTableCommand( extends RunnableCommand { override protected def innerChildren: Seq[QueryPlan[_]] = { - plan.map(Seq(_)).getOrElse(Seq.empty) + plan.toSeq } override def run(sparkSession: SparkSession): Seq[Row] = {