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 232ca4358865af0ee0179e94bfa85756bb2fb993..3d0e016a091cf6066e70bc1dfb8a6750befdde31 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 @@ -408,8 +408,11 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } } - /** Returns the name of this type of TreeNode. Defaults to the class name. */ - def nodeName: String = getClass.getSimpleName + /** + * Returns the name of this type of TreeNode. Defaults to the class name. + * Note that we remove the "Exec" suffix for physical operators here. + */ + def nodeName: String = getClass.getSimpleName.replaceAll("Exec$", "") /** * The arguments that should be included in the arg string. Defaults to the `productIterator`. @@ -426,7 +429,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case other => other :: Nil }.mkString(", ") - /** String representation of this node without any children */ + /** String representation of this node without any children. */ def simpleString: String = s"$nodeName $argString".trim override def toString: String = treeString diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 12d03a7df8c53979f738f70473b728b056cf90e5..b3a197cd96e3726800aec223a9e2902bf322f72c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -100,10 +100,10 @@ private[sql] case class LogicalRDD( } /** Physical plan node for scanning data from an RDD. */ -private[sql] case class PhysicalRDD( +private[sql] case class RDDScanExec( output: Seq[Attribute], rdd: RDD[InternalRow], - override val nodeName: String) extends LeafNode { + override val nodeName: String) extends LeafExecNode { private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) @@ -124,7 +124,7 @@ private[sql] case class PhysicalRDD( } } -private[sql] trait DataSourceScan extends LeafNode { +private[sql] trait DataSourceScanExec extends LeafExecNode { val rdd: RDD[InternalRow] val relation: BaseRelation @@ -132,19 +132,19 @@ private[sql] trait DataSourceScan extends LeafNode { // Ignore rdd when checking results override def sameResult(plan: SparkPlan): Boolean = plan match { - case other: DataSourceScan => relation == other.relation && metadata == other.metadata + case other: DataSourceScanExec => relation == other.relation && metadata == other.metadata case _ => false } } /** Physical plan node for scanning data from a relation. */ -private[sql] case class RowDataSourceScan( +private[sql] case class RowDataSourceScanExec( output: Seq[Attribute], rdd: RDD[InternalRow], @transient relation: BaseRelation, override val outputPartitioning: Partitioning, override val metadata: Map[String, String] = Map.empty) - extends DataSourceScan with CodegenSupport { + extends DataSourceScanExec with CodegenSupport { private[sql] override lazy val metrics = Map("numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) @@ -207,13 +207,13 @@ private[sql] case class RowDataSourceScan( } /** Physical plan node for scanning data from a batched relation. */ -private[sql] case class BatchedDataSourceScan( +private[sql] case class BatchedDataSourceScanExec( output: Seq[Attribute], rdd: RDD[InternalRow], @transient relation: BaseRelation, override val outputPartitioning: Partitioning, override val metadata: Map[String, String] = Map.empty) - extends DataSourceScan with CodegenSupport { + extends DataSourceScanExec with CodegenSupport { private[sql] override lazy val metrics = Map("numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"), @@ -316,7 +316,7 @@ private[sql] case class BatchedDataSourceScan( } } -private[sql] object DataSourceScan { +private[sql] object DataSourceScanExec { // Metadata keys val INPUT_PATHS = "InputPaths" val PUSHED_FILTERS = "PushedFilters" @@ -325,7 +325,7 @@ private[sql] object DataSourceScan { output: Seq[Attribute], rdd: RDD[InternalRow], relation: BaseRelation, - metadata: Map[String, String] = Map.empty): DataSourceScan = { + metadata: Map[String, String] = Map.empty): DataSourceScanExec = { val outputPartitioning = { val bucketSpec = relation match { // TODO: this should be closer to bucket planning. @@ -349,9 +349,9 @@ private[sql] object DataSourceScan { relation match { case r: HadoopFsRelation if r.fileFormat.supportBatch(r.sqlContext, relation.schema) => - BatchedDataSourceScan(output, rdd, relation, outputPartitioning, metadata) + BatchedDataSourceScanExec(output, rdd, relation, outputPartitioning, metadata) case _ => - RowDataSourceScan(output, rdd, relation, outputPartitioning, metadata) + RowDataSourceScanExec(output, rdd, relation, outputPartitioning, metadata) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala similarity index 99% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala index 3966af542e397da2b7ae80ad3895ad8c41cd3bbc..7c4756663a6e0d9ab0f130d22fa95ca03e08ac0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala @@ -33,11 +33,11 @@ import org.apache.spark.sql.execution.metric.SQLMetrics * @param output The output Schema * @param child Child operator */ -case class Expand( +case class ExpandExec( projections: Seq[Seq[Expression]], output: Seq[Attribute], child: SparkPlan) - extends UnaryNode with CodegenSupport { + extends UnaryExecNode with CodegenSupport { private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index 9938d2169f1c3227c9ed050a739b0b8ce0ce23f4..10cfec3330a2dc6c9609655b55f542a7f3720fbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -47,13 +47,13 @@ private[execution] sealed case class LazyIterator(func: () => TraversableOnce[In * @param output the output attributes of this node, which constructed in analysis phase, * and we can not change it, as the parent node bound with it already. */ -case class Generate( +case class GenerateExec( generator: Generator, join: Boolean, outer: Boolean, output: Seq[Attribute], child: SparkPlan) - extends UnaryNode { + extends UnaryExecNode { private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala similarity index 95% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala index f8aec9e7a1d1bd19cb8bb2d525ebfe932f20fded..4ab447a47b2c9ad7bf08c48be5286e8f3d870449 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala @@ -26,9 +26,9 @@ import org.apache.spark.sql.execution.metric.SQLMetrics /** * Physical plan node for scanning data from a local collection. */ -private[sql] case class LocalTableScan( +private[sql] case class LocalTableScanExec( output: Seq[Attribute], - rows: Seq[InternalRow]) extends LeafNode { + rows: Seq[InternalRow]) extends LeafExecNode { private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) 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 a444a7030256e5e5e6be2a5ecb56f08cf019e777..bb83676b7d6f929a84b93ea11c0db047d0a15f8f 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 @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker 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 -import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommand, HiveNativeCommand} +import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec, HiveNativeCommand} import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BinaryType, DateType, DecimalType, TimestampType, _} @@ -107,7 +107,7 @@ class QueryExecution(val sqlContext: SQLContext, val logical: LogicalPlan) { * execution is simply passed back to Hive. */ def hiveResultString(): Seq[String] = executedPlan match { - case ExecutedCommand(desc: DescribeTableCommand) => + case ExecutedCommandExec(desc: DescribeTableCommand) => // If it is a describe command for a Hive table, we want to have the output format // be similar with Hive. desc.run(sqlContext).map { @@ -117,7 +117,7 @@ class QueryExecution(val sqlContext: SQLContext, val logical: LogicalPlan) { .map(s => String.format(s"%-20s", s)) .mkString("\t") } - case command: ExecutedCommand => + case command: ExecutedCommandExec => command.executeCollect().map(_.getString(0)) case other => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Sort.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala similarity index 99% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/Sort.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index 04a39a126ee7b045208687cbb2a69ad83eb20839..0e4d6d72c6b5bb9d204be6f85b092139914236d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Sort.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -36,12 +36,12 @@ import org.apache.spark.util.collection.unsafe.sort.RadixSort; * @param testSpillFrequency Method for configuring periodic spilling in unit tests. If set, will * spill every `frequency` records. */ -case class Sort( +case class SortExec( sortOrder: Seq[SortOrder], global: Boolean, child: SparkPlan, testSpillFrequency: Int = 0) - extends UnaryNode with CodegenSupport { + extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 64d89f238ca79e56a56317c8b85a3bd3e62e3360..e28e456662015a9ed013149b3dd3dc327b612f7a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -41,6 +41,8 @@ import org.apache.spark.util.ThreadUtils /** * The base class for physical operators. + * + * The naming convention is that physical operators end with "Exec" suffix, e.g. [[ProjectExec]]. */ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializable { @@ -392,19 +394,19 @@ object SparkPlan { ThreadUtils.newDaemonCachedThreadPool("subquery", 16)) } -private[sql] trait LeafNode extends SparkPlan { +private[sql] trait LeafExecNode extends SparkPlan { override def children: Seq[SparkPlan] = Nil override def producedAttributes: AttributeSet = outputSet } -object UnaryNode { +object UnaryExecNode { def unapply(a: Any): Option[(SparkPlan, SparkPlan)] = a match { case s: SparkPlan if s.children.size == 1 => Some((s, s.children.head)) case _ => None } } -private[sql] trait UnaryNode extends SparkPlan { +private[sql] trait UnaryExecNode extends SparkPlan { def child: SparkPlan override def children: Seq[SparkPlan] = child :: Nil @@ -412,7 +414,7 @@ private[sql] trait UnaryNode extends SparkPlan { override def outputPartitioning: Partitioning = child.outputPartitioning } -private[sql] trait BinaryNode extends SparkPlan { +private[sql] trait BinaryExecNode extends SparkPlan { def left: SparkPlan def right: SparkPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 247f55da1d2a0b1d98d969efe720d2cce3c7e1c9..cb4b1cfeb9ba382f023ffc77953337541a3d42cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.execution.exchange.ReusedExchange +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.SQLMetricInfo import org.apache.spark.util.Utils @@ -51,7 +51,7 @@ private[sql] object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { val children = plan match { - case ReusedExchange(_, child) => child :: Nil + case ReusedExchangeExec(_, child) => child :: Nil case _ => plan.children ++ plan.subqueries } val metrics = plan.metrics.toSeq.map { case (key, metric) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala index 8d05ae470dec1deabf9a5e383df3e615bddd32fa..0afa4c7bb9d14d37a7bc13d7cbb25d4148dd4ab2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala @@ -82,10 +82,10 @@ class SparkPlanner( // when the columns of this projection are enough to evaluate all filter conditions, // just do a scan followed by a filter, with no extra project. val scan = scanBuilder(projectList.asInstanceOf[Seq[Attribute]]) - filterCondition.map(Filter(_, scan)).getOrElse(scan) + filterCondition.map(FilterExec(_, scan)).getOrElse(scan) } else { val scan = scanBuilder((projectSet ++ filterSet).toSeq) - Project(projectList, filterCondition.map(Filter(_, scan)).getOrElse(scan)) + ProjectExec(projectList, filterCondition.map(FilterExec(_, scan)).getOrElse(scan)) } } } 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 ed6b846fcfbd4c5f9df1022b285e8e13bf5f9510..3ce5f28bf32c288b64e0d71234112fd3504b1f44 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, LogicalPlan} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution -import org.apache.spark.sql.execution.columnar.{InMemoryColumnarTableScan, InMemoryRelation} +import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.exchange.ShuffleExchange @@ -44,20 +44,22 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.ReturnAnswer(rootPlan) => rootPlan match { case logical.Limit(IntegerLiteral(limit), logical.Sort(order, true, child)) => - execution.TakeOrderedAndProject(limit, order, None, planLater(child)) :: Nil + execution.TakeOrderedAndProjectExec(limit, order, None, planLater(child)) :: Nil case logical.Limit( IntegerLiteral(limit), logical.Project(projectList, logical.Sort(order, true, child))) => - execution.TakeOrderedAndProject(limit, order, Some(projectList), planLater(child)) :: Nil + execution.TakeOrderedAndProjectExec( + limit, order, Some(projectList), planLater(child)) :: Nil case logical.Limit(IntegerLiteral(limit), child) => - execution.CollectLimit(limit, planLater(child)) :: Nil + execution.CollectLimitExec(limit, planLater(child)) :: Nil case other => planLater(other) :: Nil } case logical.Limit(IntegerLiteral(limit), logical.Sort(order, true, child)) => - execution.TakeOrderedAndProject(limit, order, None, planLater(child)) :: Nil + execution.TakeOrderedAndProjectExec(limit, order, None, planLater(child)) :: Nil case logical.Limit( IntegerLiteral(limit), logical.Project(projectList, logical.Sort(order, true, child))) => - execution.TakeOrderedAndProject(limit, order, Some(projectList), planLater(child)) :: Nil + execution.TakeOrderedAndProjectExec( + limit, order, Some(projectList), planLater(child)) :: Nil case _ => Nil } } @@ -66,12 +68,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case ExtractEquiJoinKeys( LeftExistence(jt), leftKeys, rightKeys, condition, left, CanBroadcast(right)) => - Seq(joins.BroadcastHashJoin( + Seq(joins.BroadcastHashJoinExec( leftKeys, rightKeys, jt, BuildRight, condition, planLater(left), planLater(right))) // Find left semi joins where at least some predicates can be evaluated by matching join keys case ExtractEquiJoinKeys( LeftExistence(jt), leftKeys, rightKeys, condition, left, right) => - Seq(joins.ShuffledHashJoin( + Seq(joins.ShuffledHashJoinExec( leftKeys, rightKeys, jt, BuildRight, condition, planLater(left), planLater(right))) case _ => Nil } @@ -146,11 +148,11 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // --- Inner joins -------------------------------------------------------------------------- case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, CanBroadcast(right)) => - Seq(joins.BroadcastHashJoin( + Seq(joins.BroadcastHashJoinExec( leftKeys, rightKeys, Inner, BuildRight, condition, planLater(left), planLater(right))) case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, CanBroadcast(left), right) => - Seq(joins.BroadcastHashJoin( + Seq(joins.BroadcastHashJoinExec( leftKeys, rightKeys, Inner, BuildLeft, condition, planLater(left), planLater(right))) case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) @@ -162,41 +164,41 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } else { BuildLeft } - Seq(joins.ShuffledHashJoin( + Seq(joins.ShuffledHashJoinExec( leftKeys, rightKeys, Inner, buildSide, condition, planLater(left), planLater(right))) case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) if RowOrdering.isOrderable(leftKeys) => - joins.SortMergeJoin( + joins.SortMergeJoinExec( leftKeys, rightKeys, Inner, condition, planLater(left), planLater(right)) :: Nil // --- Outer joins -------------------------------------------------------------------------- case ExtractEquiJoinKeys( LeftOuter, leftKeys, rightKeys, condition, left, CanBroadcast(right)) => - Seq(joins.BroadcastHashJoin( + Seq(joins.BroadcastHashJoinExec( leftKeys, rightKeys, LeftOuter, BuildRight, condition, planLater(left), planLater(right))) case ExtractEquiJoinKeys( RightOuter, leftKeys, rightKeys, condition, CanBroadcast(left), right) => - Seq(joins.BroadcastHashJoin( + Seq(joins.BroadcastHashJoinExec( leftKeys, rightKeys, RightOuter, BuildLeft, condition, planLater(left), planLater(right))) case ExtractEquiJoinKeys(LeftOuter, leftKeys, rightKeys, condition, left, right) if !conf.preferSortMergeJoin && canBuildHashMap(right) && muchSmaller(right, left) || !RowOrdering.isOrderable(leftKeys) => - Seq(joins.ShuffledHashJoin( + Seq(joins.ShuffledHashJoinExec( leftKeys, rightKeys, LeftOuter, BuildRight, condition, planLater(left), planLater(right))) case ExtractEquiJoinKeys(RightOuter, leftKeys, rightKeys, condition, left, right) if !conf.preferSortMergeJoin && canBuildHashMap(left) && muchSmaller(left, right) || !RowOrdering.isOrderable(leftKeys) => - Seq(joins.ShuffledHashJoin( + Seq(joins.ShuffledHashJoinExec( leftKeys, rightKeys, RightOuter, BuildLeft, condition, planLater(left), planLater(right))) case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) if RowOrdering.isOrderable(leftKeys) => - joins.SortMergeJoin( + joins.SortMergeJoinExec( leftKeys, rightKeys, joinType, condition, planLater(left), planLater(right)) :: Nil // --- Cases where this strategy does not apply --------------------------------------------- @@ -278,10 +280,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object BroadcastNestedLoop extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case j @ logical.Join(CanBroadcast(left), right, Inner | RightOuter, condition) => - execution.joins.BroadcastNestedLoopJoin( + execution.joins.BroadcastNestedLoopJoinExec( planLater(left), planLater(right), joins.BuildLeft, j.joinType, condition) :: Nil case j @ logical.Join(left, CanBroadcast(right), Inner | LeftOuter | LeftSemi, condition) => - execution.joins.BroadcastNestedLoopJoin( + execution.joins.BroadcastNestedLoopJoinExec( planLater(left), planLater(right), joins.BuildRight, j.joinType, condition) :: Nil case _ => Nil } @@ -290,10 +292,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object CartesianProduct extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Join(left, right, Inner, None) => - execution.joins.CartesianProduct(planLater(left), planLater(right)) :: Nil + execution.joins.CartesianProductExec(planLater(left), planLater(right)) :: Nil case logical.Join(left, right, Inner, Some(condition)) => - execution.Filter(condition, - execution.joins.CartesianProduct(planLater(left), planLater(right))) :: Nil + execution.FilterExec(condition, + execution.joins.CartesianProductExec(planLater(left), planLater(right))) :: Nil case _ => Nil } } @@ -308,7 +310,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { joins.BuildLeft } // This join could be very slow or even hang forever - joins.BroadcastNestedLoopJoin( + joins.BroadcastNestedLoopJoinExec( planLater(left), planLater(right), buildSide, joinType, condition) :: Nil case _ => Nil } @@ -323,7 +325,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { projectList, filters, identity[Seq[Expression]], // All filters still need to be evaluated. - InMemoryColumnarTableScan(_, filters, mem)) :: Nil + InMemoryTableScanExec(_, filters, mem)) :: Nil case _ => Nil } } @@ -333,11 +335,11 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def numPartitions: Int = self.numPartitions def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case r: RunnableCommand => ExecutedCommand(r) :: Nil + case r: RunnableCommand => ExecutedCommandExec(r) :: Nil case MemoryPlan(sink, output) => val encoder = RowEncoder(sink.schema) - LocalTableScan(output, sink.allData.map(r => encoder.toRow(r).copy())) :: Nil + LocalTableScanExec(output, sink.allData.map(r => encoder.toRow(r).copy())) :: Nil case logical.Distinct(child) => throw new IllegalStateException( @@ -349,19 +351,19 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.DeserializeToObject(deserializer, objAttr, child) => execution.DeserializeToObject(deserializer, objAttr, planLater(child)) :: Nil case logical.SerializeFromObject(serializer, child) => - execution.SerializeFromObject(serializer, planLater(child)) :: Nil + execution.SerializeFromObjectExec(serializer, planLater(child)) :: Nil case logical.MapPartitions(f, objAttr, child) => - execution.MapPartitions(f, objAttr, planLater(child)) :: Nil + execution.MapPartitionsExec(f, objAttr, planLater(child)) :: Nil case logical.MapElements(f, objAttr, child) => - execution.MapElements(f, objAttr, planLater(child)) :: Nil + execution.MapElementsExec(f, objAttr, planLater(child)) :: Nil case logical.AppendColumns(f, in, out, child) => - execution.AppendColumns(f, in, out, planLater(child)) :: Nil + execution.AppendColumnsExec(f, in, out, planLater(child)) :: Nil case logical.AppendColumnsWithObject(f, childSer, newSer, child) => - execution.AppendColumnsWithObject(f, childSer, newSer, planLater(child)) :: Nil + execution.AppendColumnsWithObjectExec(f, childSer, newSer, planLater(child)) :: Nil case logical.MapGroups(f, key, value, grouping, data, objAttr, child) => - execution.MapGroups(f, key, value, grouping, data, objAttr, planLater(child)) :: Nil + execution.MapGroupsExec(f, key, value, grouping, data, objAttr, planLater(child)) :: Nil case logical.CoGroup(f, key, lObj, rObj, lGroup, rGroup, lAttr, rAttr, oAttr, left, right) => - execution.CoGroup( + execution.CoGroupExec( f, key, lObj, rObj, lGroup, rGroup, lAttr, rAttr, oAttr, planLater(left), planLater(right)) :: Nil @@ -369,45 +371,45 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { if (shuffle) { ShuffleExchange(RoundRobinPartitioning(numPartitions), planLater(child)) :: Nil } else { - execution.Coalesce(numPartitions, planLater(child)) :: Nil + execution.CoalesceExec(numPartitions, planLater(child)) :: Nil } case logical.SortPartitions(sortExprs, child) => // This sort only sorts tuples within a partition. Its requiredDistribution will be // an UnspecifiedDistribution. - execution.Sort(sortExprs, global = false, child = planLater(child)) :: Nil + execution.SortExec(sortExprs, global = false, child = planLater(child)) :: Nil case logical.Sort(sortExprs, global, child) => - execution.Sort(sortExprs, global, planLater(child)) :: Nil + execution.SortExec(sortExprs, global, planLater(child)) :: Nil case logical.Project(projectList, child) => - execution.Project(projectList, planLater(child)) :: Nil + execution.ProjectExec(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => - execution.Filter(condition, planLater(child)) :: Nil + execution.FilterExec(condition, planLater(child)) :: Nil case e @ logical.Expand(_, _, child) => - execution.Expand(e.projections, e.output, planLater(child)) :: Nil + execution.ExpandExec(e.projections, e.output, planLater(child)) :: Nil case logical.Window(windowExprs, partitionSpec, orderSpec, child) => - execution.Window(windowExprs, partitionSpec, orderSpec, planLater(child)) :: Nil + execution.WindowExec(windowExprs, partitionSpec, orderSpec, planLater(child)) :: Nil case logical.Sample(lb, ub, withReplacement, seed, child) => - execution.Sample(lb, ub, withReplacement, seed, planLater(child)) :: Nil + execution.SampleExec(lb, ub, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => - LocalTableScan(output, data) :: Nil + LocalTableScanExec(output, data) :: Nil case logical.LocalLimit(IntegerLiteral(limit), child) => - execution.LocalLimit(limit, planLater(child)) :: Nil + execution.LocalLimitExec(limit, planLater(child)) :: Nil case logical.GlobalLimit(IntegerLiteral(limit), child) => - execution.GlobalLimit(limit, planLater(child)) :: Nil + execution.GlobalLimitExec(limit, planLater(child)) :: Nil case logical.Union(unionChildren) => - execution.Union(unionChildren.map(planLater)) :: Nil + execution.UnionExec(unionChildren.map(planLater)) :: Nil case logical.Except(left, right) => - execution.Except(planLater(left), planLater(right)) :: Nil + execution.ExceptExec(planLater(left), planLater(right)) :: Nil case g @ logical.Generate(generator, join, outer, _, _, child) => - execution.Generate( + execution.GenerateExec( generator, join = join, outer = outer, g.output, planLater(child)) :: Nil case logical.OneRowRelation => - execution.PhysicalRDD(Nil, singleRowRdd, "OneRowRelation") :: Nil + execution.RDDScanExec(Nil, singleRowRdd, "OneRowRelation") :: Nil case r @ logical.Range(start, end, step, numSlices, output) => - execution.Range(start, step, numSlices, r.numElements, output) :: Nil + execution.RangeExec(start, step, numSlices, r.numElements, output) :: Nil case logical.RepartitionByExpression(expressions, child, nPartitions) => exchange.ShuffleExchange(HashPartitioning( expressions, nPartitions.getOrElse(numPartitions)), planLater(child)) :: Nil - case LogicalRDD(output, rdd) => PhysicalRDD(output, rdd, "ExistingRDD") :: Nil + case LogicalRDD(output, rdd) => RDDScanExec(output, rdd, "ExistingRDD") :: Nil case BroadcastHint(child) => planLater(child) :: Nil case _ => Nil } @@ -416,7 +418,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object DDLStrategy extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case CreateTableUsing(tableIdent, userSpecifiedSchema, provider, true, opts, false, _) => - ExecutedCommand( + ExecutedCommandExec( CreateTempTableUsing( tableIdent, userSpecifiedSchema, provider, opts)) :: Nil case c: CreateTableUsing if !c.temporary => @@ -430,15 +432,15 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case c: CreateTableUsingAsSelect if c.temporary => val cmd = CreateTempTableUsingAsSelect( c.tableIdent, c.provider, Array.empty[String], c.mode, c.options, c.child) - ExecutedCommand(cmd) :: Nil + ExecutedCommandExec(cmd) :: Nil case c: CreateTableUsingAsSelect if !c.temporary => sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") case logical.ShowFunctions(db, pattern) => - ExecutedCommand(ShowFunctions(db, pattern)) :: Nil + ExecutedCommandExec(ShowFunctions(db, pattern)) :: Nil case logical.DescribeFunction(function, extended) => - ExecutedCommand(DescribeFunction(function, extended)) :: Nil + ExecutedCommandExec(DescribeFunction(function, extended)) :: Nil case _ => Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 944962b1c8844b25d467de761372e5a4bda6e50c..6a03bd08c547e8cdb4df8e336ce6944195900cc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.toCommentSafeString import org.apache.spark.sql.execution.aggregate.TungstenAggregate -import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, SortMergeJoin} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.{LongSQLMetricValue, SQLMetrics} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -39,10 +39,10 @@ trait CodegenSupport extends SparkPlan { /** Prefix used in the current operator's variable names. */ private def variablePrefix: String = this match { case _: TungstenAggregate => "agg" - case _: BroadcastHashJoin => "bhj" - case _: SortMergeJoin => "smj" - case _: PhysicalRDD => "rdd" - case _: DataSourceScan => "scan" + case _: BroadcastHashJoinExec => "bhj" + case _: SortMergeJoinExec => "smj" + case _: RDDScanExec => "rdd" + case _: DataSourceScanExec => "scan" case _ => nodeName.toLowerCase } @@ -219,7 +219,7 @@ trait CodegenSupport extends SparkPlan { * This is the leaf node of a tree with WholeStageCodegen, is used to generate code that consumes * an RDD iterator of InternalRow. */ -case class InputAdapter(child: SparkPlan) extends UnaryNode with CodegenSupport { +case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning @@ -256,7 +256,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryNode with CodegenSupport override def treeChildren: Seq[SparkPlan] = Nil } -object WholeStageCodegen { +object WholeStageCodegenExec { val PIPELINE_DURATION_METRIC = "duration" } @@ -288,7 +288,7 @@ object WholeStageCodegen { * doCodeGen() will create a CodeGenContext, which will hold a list of variables for input, * used to generated code for BoundReference. */ -case class WholeStageCodegen(child: SparkPlan) extends UnaryNode with CodegenSupport { +case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning @@ -296,7 +296,7 @@ case class WholeStageCodegen(child: SparkPlan) extends UnaryNode with CodegenSup override private[sql] lazy val metrics = Map( "pipelineTime" -> SQLMetrics.createTimingMetric(sparkContext, - WholeStageCodegen.PIPELINE_DURATION_METRIC)) + WholeStageCodegenExec.PIPELINE_DURATION_METRIC)) /** * Generates code for this subtree. @@ -457,7 +457,7 @@ case class CollapseCodegenStages(conf: SQLConf) extends Rule[SparkPlan] { * Inserts a InputAdapter on top of those that do not support codegen. */ private def insertInputAdapter(plan: SparkPlan): SparkPlan = plan match { - case j @ SortMergeJoin(_, _, _, _, left, right) if j.supportCodegen => + case j @ SortMergeJoinExec(_, _, _, _, left, right) if j.supportCodegen => // The children of SortMergeJoin should do codegen separately. j.copy(left = InputAdapter(insertWholeStageCodegen(left)), right = InputAdapter(insertWholeStageCodegen(right))) @@ -477,7 +477,7 @@ case class CollapseCodegenStages(conf: SQLConf) extends Rule[SparkPlan] { case plan if plan.output.length == 1 && plan.output.head.dataType.isInstanceOf[ObjectType] => plan.withNewChildren(plan.children.map(insertWholeStageCodegen)) case plan: CodegenSupport if supportCodegen(plan) => - WholeStageCodegen(insertInputAdapter(plan)) + WholeStageCodegenExec(insertInputAdapter(plan)) case other => other.withNewChildren(other.children.map(insertWholeStageCodegen)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala similarity index 99% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala index a46d0e0ba7afa9924e7c9554c6507a87161fd947..97bbab65af1ded25eba9a252b8246c541cd166d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala @@ -80,12 +80,12 @@ import org.apache.spark.util.collection.unsafe.sort.{UnsafeExternalSorter, Unsaf * Entire Partition, Sliding, Growing & Shrinking. Boundary evaluation is also delegated to a pair * of specialized classes: [[RowBoundOrdering]] & [[RangeBoundOrdering]]. */ -case class Window( +case class WindowExec( windowExpression: Seq[NamedExpression], partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], child: SparkPlan) - extends UnaryNode { + extends UnaryExecNode { override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala similarity index 97% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala index 9fcfea8381ac4fa42b1ef20acc1d12799c446a65..3169e0a2fd86a34018e2572eace0ef055a16a3c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala @@ -23,10 +23,10 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, UnspecifiedDistribution} -import org.apache.spark.sql.execution.{SparkPlan, UnaryNode} +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics -case class SortBasedAggregate( +case class SortBasedAggregateExec( requiredChildDistributionExpressions: Option[Seq[Expression]], groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], @@ -34,7 +34,7 @@ case class SortBasedAggregate( initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], child: SparkPlan) - extends UnaryNode { + extends UnaryExecNode { private[this] val aggregateBufferAttributes = { aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index 49b682a951f62ab112ed601410615b8954bda57a..782da0ea604d335c5acc45b7a29a333dbc70c79e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -38,7 +38,7 @@ case class TungstenAggregate( initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], child: SparkPlan) - extends UnaryNode with CodegenSupport { + extends UnaryExecNode with CodegenSupport { private[this] val aggregateBufferAttributes = { aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala index 4682949fa1c7d2adbaec413521fb368cef26b0cc..f93c446007422c8bc0c98a8db0ded7dd704d1269 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.aggregate import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.streaming.{StateStoreRestore, StateStoreSave} +import org.apache.spark.sql.execution.streaming.{StateStoreRestoreExec, StateStoreSaveExec} /** * Utility functions used by the query planner to convert our plan to new aggregation code path. @@ -35,7 +35,7 @@ object Utils { val completeAggregateExpressions = aggregateExpressions.map(_.copy(mode = Complete)) val completeAggregateAttributes = completeAggregateExpressions.map(_.resultAttribute) - SortBasedAggregate( + SortBasedAggregateExec( requiredChildDistributionExpressions = Some(groupingExpressions), groupingExpressions = groupingExpressions, aggregateExpressions = completeAggregateExpressions, @@ -66,7 +66,7 @@ object Utils { resultExpressions = resultExpressions, child = child) } else { - SortBasedAggregate( + SortBasedAggregateExec( requiredChildDistributionExpressions = requiredChildDistributionExpressions, groupingExpressions = groupingExpressions, aggregateExpressions = aggregateExpressions, @@ -295,7 +295,7 @@ object Utils { child = partialAggregate) } - val restored = StateStoreRestore(groupingAttributes, None, partialMerged1) + val restored = StateStoreRestoreExec(groupingAttributes, None, partialMerged1) val partialMerged2: SparkPlan = { val aggregateExpressions = functionsWithoutDistinct.map(_.copy(mode = PartialMerge)) @@ -312,7 +312,7 @@ object Utils { child = restored) } - val saved = StateStoreSave(groupingAttributes, None, partialMerged2) + val saved = StateStoreSaveExec(groupingAttributes, None, partialMerged2) val finalAndCompleteAggregate: SparkPlan = { val finalAggregateExpressions = functionsWithoutDistinct.map(_.copy(mode = Final)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 892c57ae7d7c9ff8f8d2248ebe14d81defcf9c82..83f527f5551acd8a6a9da2681685f0fb1d619525 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -20,14 +20,15 @@ package org.apache.spark.sql.execution import org.apache.spark.rdd.{PartitionwiseSampledRDD, RDD} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, ExpressionCanonicalizer, GenerateUnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, ExpressionCanonicalizer} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.LongType import org.apache.spark.util.random.{BernoulliCellSampler, PoissonSampler} -case class Project(projectList: Seq[NamedExpression], child: SparkPlan) - extends UnaryNode with CodegenSupport { +/** Physical plan for Project. */ +case class ProjectExec(projectList: Seq[NamedExpression], child: SparkPlan) + extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = projectList.map(_.toAttribute) @@ -74,8 +75,9 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) } -case class Filter(condition: Expression, child: SparkPlan) - extends UnaryNode with CodegenSupport with PredicateHelper { +/** Physical plan for Filter. */ +case class FilterExec(condition: Expression, child: SparkPlan) + extends UnaryExecNode with CodegenSupport with PredicateHelper { // Split out all the IsNotNulls from condition. private val (notNullPreds, otherPreds) = splitConjunctivePredicates(condition).partition { @@ -209,7 +211,7 @@ case class Filter(condition: Expression, child: SparkPlan) } /** - * Sample the dataset. + * Physical plan for sampling the dataset. * * @param lowerBound Lower-bound of the sampling probability (usually 0.0) * @param upperBound Upper-bound of the sampling probability. The expected fraction sampled @@ -218,12 +220,12 @@ case class Filter(condition: Expression, child: SparkPlan) * @param seed the random seed * @param child the SparkPlan */ -case class Sample( +case class SampleExec( lowerBound: Double, upperBound: Double, withReplacement: Boolean, seed: Long, - child: SparkPlan) extends UnaryNode with CodegenSupport { + child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output private[sql] override lazy val metrics = Map( @@ -301,13 +303,23 @@ case class Sample( } } -case class Range( + +/** + * Physical plan for range (generating a range of 64 bit numbers. + * + * @param start first number in the range, inclusive. + * @param step size of the step increment. + * @param numSlices number of partitions. + * @param numElements total number of elements to output. + * @param output output attributes. + */ +case class RangeExec( start: Long, step: Long, numSlices: Int, numElements: BigInt, output: Seq[Attribute]) - extends LeafNode with CodegenSupport { + extends LeafExecNode with CodegenSupport { private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) @@ -449,9 +461,9 @@ case class Range( } /** - * Union two plans, without a distinct. This is UNION ALL in SQL. + * Physical plan for unioning two plans, without a distinct. This is UNION ALL in SQL. */ -case class Union(children: Seq[SparkPlan]) extends SparkPlan { +case class UnionExec(children: Seq[SparkPlan]) extends SparkPlan { override def output: Seq[Attribute] = children.map(_.output).transpose.map(attrs => attrs.head.withNullability(attrs.exists(_.nullable))) @@ -461,12 +473,12 @@ case class Union(children: Seq[SparkPlan]) extends SparkPlan { } /** - * Return a new RDD that has exactly `numPartitions` partitions. + * Physical plan for returning a new RDD that has exactly `numPartitions` partitions. * Similar to coalesce defined on an [[RDD]], this operation results in a narrow dependency, e.g. * if you go from 1000 partitions to 100 partitions, there will not be a shuffle, instead each of * the 100 new partitions will claim 10 of the current partitions. */ -case class Coalesce(numPartitions: Int, child: SparkPlan) extends UnaryNode { +case class CoalesceExec(numPartitions: Int, child: SparkPlan) extends UnaryExecNode { override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = { @@ -480,10 +492,10 @@ case class Coalesce(numPartitions: Int, child: SparkPlan) extends UnaryNode { } /** - * Returns a table with the elements from left that are not in right using + * Physical plan for returning a table with the elements from left that are not in right using * the built-in spark subtract function. */ -case class Except(left: SparkPlan, right: SparkPlan) extends BinaryNode { +case class ExceptExec(left: SparkPlan, right: SparkPlan) extends BinaryExecNode { override def output: Seq[Attribute] = left.output protected override def doExecute(): RDD[InternalRow] = { @@ -496,18 +508,18 @@ case class Except(left: SparkPlan, right: SparkPlan) extends BinaryNode { * (hopefully structurally equivalent) tree from a different optimization sequence into an already * resolved tree. */ -case class OutputFaker(output: Seq[Attribute], child: SparkPlan) extends SparkPlan { +case class OutputFakerExec(output: Seq[Attribute], child: SparkPlan) extends SparkPlan { def children: Seq[SparkPlan] = child :: Nil protected override def doExecute(): RDD[InternalRow] = child.execute() } /** - * A plan as subquery. + * Physical plan for a subquery. * * This is used to generate tree string for SparkScalarSubquery. */ -case class Subquery(name: String, child: SparkPlan) extends UnaryNode { +case class SubqueryExec(name: String, child: SparkPlan) extends UnaryExecNode { override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning override def outputOrdering: Seq[SortOrder] = child.outputOrdering diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarTableScan.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 1f964b1fc1dce23a0f2ee52032f926df5edc133c..cb957b9666f550c5aead5afe7dae6799579c172c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -29,7 +29,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.execution.{LeafNode, SparkPlan} +import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType import org.apache.spark.storage.StorageLevel @@ -210,11 +210,11 @@ private[sql] case class InMemoryRelation( } } -private[sql] case class InMemoryColumnarTableScan( +private[sql] case class InMemoryTableScanExec( attributes: Seq[Attribute], predicates: Seq[Expression], @transient relation: InMemoryRelation) - extends LeafNode { + extends LeafExecNode { private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) 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 45a32131b65dbe58a4088df9695b89c64065c46e..971770a97b5247e5bccbd4a00ffa7c0adbc416ac 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 @@ -42,7 +42,7 @@ private[sql] trait RunnableCommand extends LogicalPlan with logical.Command { * A physical operator that executes the run method of a `RunnableCommand` and * saves the result to prevent multiple executions. */ -private[sql] case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { +private[sql] case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkPlan { /** * A concrete command should override this lazy field to wrap up any side effects caused by the * command or any other computation that should be evaluated exactly once. The value of this field diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index ac3c52e9017959ebea9cd2db3d59aac258cf3233..9bebd74b4b3a2ea738c0522698f010c34d7587c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -32,8 +32,8 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.DataSourceScan.{INPUT_PATHS, PUSHED_FILTERS} -import org.apache.spark.sql.execution.command.ExecutedCommand +import org.apache.spark.sql.execution.DataSourceScanExec.{INPUT_PATHS, PUSHED_FILTERS} +import org.apache.spark.sql.execution.command.ExecutedCommandExec import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -105,12 +105,12 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { (a, _) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray))) :: Nil case l @ LogicalRelation(baseRelation: TableScan, _, _) => - execution.DataSourceScan.create( + execution.DataSourceScanExec.create( l.output, toCatalystRDD(l, baseRelation.buildScan()), baseRelation) :: Nil case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _, _), part, query, overwrite, false) if part.isEmpty => - ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil + ExecutedCommandExec(InsertIntoDataSource(l, query, overwrite)) :: Nil case _ => Nil } @@ -214,22 +214,22 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { // Don't request columns that are only referenced by pushed filters. .filterNot(handledSet.contains) - val scan = execution.DataSourceScan.create( + val scan = execution.DataSourceScanExec.create( projects.map(_.toAttribute), scanBuilder(requestedColumns, candidatePredicates, pushedFilters), relation.relation, metadata) - filterCondition.map(execution.Filter(_, scan)).getOrElse(scan) + filterCondition.map(execution.FilterExec(_, scan)).getOrElse(scan) } else { // Don't request columns that are only referenced by pushed filters. val requestedColumns = (projectSet ++ filterSet -- handledSet).map(relation.attributeMap).toSeq - val scan = execution.DataSourceScan.create( + val scan = execution.DataSourceScanExec.create( requestedColumns, scanBuilder(requestedColumns, candidatePredicates, pushedFilters), relation.relation, metadata) - execution.Project( - projects, filterCondition.map(execution.Filter(_, scan)).getOrElse(scan)) + execution.ProjectExec( + projects, filterCondition.map(execution.FilterExec(_, scan)).getOrElse(scan)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index c1a97de72f65dcd521c07cd1b6daefb980c803d2..751daa0fe2d00a27bd39e3607e76948f0bca95ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.{DataSourceScan, SparkPlan} +import org.apache.spark.sql.execution.{DataSourceScanExec, SparkPlan} /** * A strategy for planning scans over collections of files that might be partitioned or bucketed @@ -192,7 +192,7 @@ private[sql] object FileSourceStrategy extends Strategy with Logging { } val scan = - DataSourceScan.create( + DataSourceScanExec.create( readDataColumns ++ partitionColumns, new FileScanRDD( files.sqlContext, @@ -205,11 +205,11 @@ private[sql] object FileSourceStrategy extends Strategy with Logging { "ReadSchema" -> prunedDataSchema.simpleString)) val afterScanFilter = afterScanFilters.toSeq.reduceOption(expressions.And) - val withFilter = afterScanFilter.map(execution.Filter(_, scan)).getOrElse(scan) + val withFilter = afterScanFilter.map(execution.FilterExec(_, scan)).getOrElse(scan) val withProjections = if (projects == withFilter.output) { withFilter } else { - execution.Project(projects, withFilter) + execution.ProjectExec(projects, withFilter) } withProjections :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index e6079ecaadc7d4915eeec04c08fca2d08d7af0d3..5b96ab10c9915dcf081abb042e4fd2e21b0d6c18 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -49,9 +49,9 @@ package object debug { } def codegenString(plan: SparkPlan): String = { - val codegenSubtrees = new collection.mutable.HashSet[WholeStageCodegen]() + val codegenSubtrees = new collection.mutable.HashSet[WholeStageCodegenExec]() plan transform { - case s: WholeStageCodegen => + case s: WholeStageCodegenExec => codegenSubtrees += s s case s => s @@ -86,11 +86,11 @@ package object debug { val debugPlan = plan transform { case s: SparkPlan if !visited.contains(new TreeNodeRef(s)) => visited += new TreeNodeRef(s) - DebugNode(s) + DebugExec(s) } debugPrint(s"Results returned: ${debugPlan.execute().count()}") debugPlan.foreach { - case d: DebugNode => d.dumpStats() + case d: DebugExec => d.dumpStats() case _ => } } @@ -104,7 +104,7 @@ package object debug { } } - private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode with CodegenSupport { + private[sql] case class DebugExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { def output: Seq[Attribute] = child.output implicit object SetAccumulatorParam extends AccumulatorParam[HashSet[String]] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala similarity index 94% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index 87a113ee1c89d913cbc3c8cba094cba3ede9e05f..573ca195ac13fb80b830c1afd9e2b886c872ae37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -30,10 +30,10 @@ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.ThreadUtils /** - * A [[BroadcastExchange]] collects, transforms and finally broadcasts the result of a transformed - * SparkPlan. + * A [[BroadcastExchangeExec]] collects, transforms and finally broadcasts the result of + * a transformed SparkPlan. */ -case class BroadcastExchange( +case class BroadcastExchangeExec( mode: BroadcastMode, child: SparkPlan) extends Exchange { @@ -46,7 +46,7 @@ case class BroadcastExchange( override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) override def sameResult(plan: SparkPlan): Boolean = plan match { - case p: BroadcastExchange => + case p: BroadcastExchangeExec => mode.compatibleWith(p.mode) && child.sameResult(p.child) case _ => false } @@ -85,7 +85,7 @@ case class BroadcastExchange( longMetric("broadcastTime") += (System.nanoTime() - beforeBroadcast) / 1000000 broadcasted } - }(BroadcastExchange.executionContext) + }(BroadcastExchangeExec.executionContext) } override protected def doPrepare(): Unit = { @@ -103,7 +103,7 @@ case class BroadcastExchange( } } -object BroadcastExchange { +object BroadcastExchangeExec { private[execution] val executionContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", 128)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 4864db7f2ac9be5882e118a9171880a23a45ddf9..446571aa8409f86d4d7f6b8b3adc7030ed2eb643 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -160,7 +160,7 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { case (child, distribution) if child.outputPartitioning.satisfies(distribution) => child case (child, BroadcastDistribution(mode)) => - BroadcastExchange(mode, child) + BroadcastExchangeExec(mode, child) case (child, distribution) => ShuffleExchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) } @@ -237,7 +237,7 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { if (requiredOrdering.nonEmpty) { // If child.outputOrdering is [a, b] and requiredOrdering is [a], we do not need to sort. if (requiredOrdering != child.outputOrdering.take(requiredOrdering.length)) { - Sort(requiredOrdering, global = false, child = child) + SortExec(requiredOrdering, global = false, child = child) } else { child } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala index df7ad48812051135adb6a6e789a2dc85724508bb..9da9df617405d8ea58a6c0b9328f42af742360d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} +import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, UnaryExecNode} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType @@ -36,7 +36,7 @@ import org.apache.spark.sql.types.StructType * differs significantly, the concept is similar to the exchange operator described in * "Volcano -- An Extensible and Parallel Query Evaluation System" by Goetz Graefe. */ -abstract class Exchange extends UnaryNode { +abstract class Exchange extends UnaryExecNode { override def output: Seq[Attribute] = child.output } @@ -45,7 +45,8 @@ abstract class Exchange extends UnaryNode { * logically identical output will have distinct sets of output attribute ids, so we need to * preserve the original ids because they're what downstream operators are expecting. */ -case class ReusedExchange(override val output: Seq[Attribute], child: Exchange) extends LeafNode { +case class ReusedExchangeExec(override val output: Seq[Attribute], child: Exchange) + extends LeafExecNode { override def sameResult(plan: SparkPlan): Boolean = { // Ignore this wrapper. `plan` could also be a ReusedExchange, so we reverse the order here. @@ -86,7 +87,7 @@ case class ReuseExchange(conf: SQLConf) extends Rule[SparkPlan] { if (samePlan.isDefined) { // Keep the output of this exchange, the following plans require that to resolve // attributes. - ReusedExchange(exchange.output, samePlan.get) + ReusedExchangeExec(exchange.output, samePlan.get) } else { sameSchema += exchange exchange diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 89487c6b871500afac3a1baab085a31dd74e6aed..51399e18301162e3021256d5acdda42b09139735 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, GenerateUnsafeProjection} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, Partitioning, UnspecifiedDistribution} -import org.apache.spark.sql.execution.{BinaryNode, CodegenSupport, SparkPlan} +import org.apache.spark.sql.execution.{BinaryExecNode, CodegenSupport, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.LongType @@ -35,7 +35,7 @@ import org.apache.spark.sql.types.LongType * broadcast relation. This data is then placed in a Spark broadcast variable. The streamed * relation is not shuffled. */ -case class BroadcastHashJoin( +case class BroadcastHashJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -43,7 +43,7 @@ case class BroadcastHashJoin( condition: Option[Expression], left: SparkPlan, right: SparkPlan) - extends BinaryNode with HashJoin with CodegenSupport { + extends BinaryExecNode with HashJoin with CodegenSupport { override private[sql] lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala index 4ba710c10a41a58ae9f55a609e9012d77197b48e..51afa0017dd268357626086a9d7a0a47c71a3e1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala @@ -23,16 +23,16 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.collection.{BitSet, CompactBuffer} -case class BroadcastNestedLoopJoin( +case class BroadcastNestedLoopJoinExec( left: SparkPlan, right: SparkPlan, buildSide: BuildSide, joinType: JoinType, - condition: Option[Expression]) extends BinaryNode { + condition: Option[Expression]) extends BinaryExecNode { override private[sql] lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala index b1de52b5f44523589f4df938252864ccbc45453d..3ce7c0e31561e6dce01d96be81f66646e6ddbad7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala @@ -22,7 +22,7 @@ import org.apache.spark.rdd.{CartesianPartition, CartesianRDD, RDD} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner -import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter @@ -79,7 +79,7 @@ class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numField } -case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { +case class CartesianProductExec(left: SparkPlan, right: SparkPlan) extends BinaryExecNode { override def output: Seq[Attribute] = left.output ++ right.output override private[sql] lazy val metrics = Map( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 785373b22590f5cbe1be0edd0b06c5033d0b84df..68cd3cb49c1609badccaa97d98a6f269d5abdfd9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -23,13 +23,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{BindReferences, Expression, UnsafeRow} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics /** * Performs a hash join of two child relations by first shuffling the data using the join keys. */ -case class ShuffledHashJoin( +case class ShuffledHashJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -37,7 +37,7 @@ case class ShuffledHashJoin( condition: Option[Expression], left: SparkPlan, right: SparkPlan) - extends BinaryNode with HashJoin { + extends BinaryExecNode with HashJoin { override private[sql] lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala similarity index 99% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 4e45fd656007f3c3e7b4ffb32575d0d65449174e..96b283a5e4d8e9dce52c8865129c4c74bfd6cb99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -25,20 +25,20 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.{BinaryNode, CodegenSupport, RowIterator, SparkPlan} +import org.apache.spark.sql.execution.{BinaryExecNode, CodegenSupport, RowIterator, SparkPlan} import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetrics} import org.apache.spark.util.collection.BitSet /** * Performs an sort merge join of two child relations. */ -case class SortMergeJoin( +case class SortMergeJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, condition: Option[Expression], left: SparkPlan, - right: SparkPlan) extends BinaryNode with CodegenSupport { + right: SparkPlan) extends BinaryExecNode with CodegenSupport { override private[sql] lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) @@ -466,7 +466,7 @@ case class SortMergeJoin( } /** - * Helper class that is used to implement [[SortMergeJoin]]. + * Helper class that is used to implement [[SortMergeJoinExec]]. * * To perform an inner (outer) join, users of this class call [[findNextInnerJoinRows()]] * ([[findNextOuterJoinRows()]]), which returns `true` if a result has been produced and `false` diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index c9a14593fb4009baa6bfdcf5348c8ef7b6b28008..b71f3335c99e5bc712b77aedb512ab2f72976911 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.execution.exchange.ShuffleExchange * This operator will be used when a logical `Limit` operation is the final operator in an * logical plan, which happens when the user is collecting results back to the driver. */ -case class CollectLimit(limit: Int, child: SparkPlan) extends UnaryNode { +case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode { override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = SinglePartition override def executeCollect(): Array[InternalRow] = child.executeTake(limit) @@ -46,9 +46,10 @@ case class CollectLimit(limit: Int, child: SparkPlan) extends UnaryNode { } /** - * Helper trait which defines methods that are shared by both [[LocalLimit]] and [[GlobalLimit]]. + * Helper trait which defines methods that are shared by both + * [[LocalLimitExec]] and [[GlobalLimitExec]]. */ -trait BaseLimit extends UnaryNode with CodegenSupport { +trait BaseLimitExec extends UnaryExecNode with CodegenSupport { val limit: Int override def output: Seq[Attribute] = child.output override def outputOrdering: Seq[SortOrder] = child.outputOrdering @@ -91,29 +92,29 @@ trait BaseLimit extends UnaryNode with CodegenSupport { /** * Take the first `limit` elements of each child partition, but do not collect or shuffle them. */ -case class LocalLimit(limit: Int, child: SparkPlan) extends BaseLimit { +case class LocalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** * Take the first `limit` elements of the child's single output partition. */ -case class GlobalLimit(limit: Int, child: SparkPlan) extends BaseLimit { +case class GlobalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { override def requiredChildDistribution: List[Distribution] = AllTuples :: Nil } /** * Take the first limit elements as defined by the sortOrder, and do projection if needed. - * This is logically equivalent to having a Limit operator after a [[Sort]] operator, - * or having a [[Project]] operator between them. + * This is logically equivalent to having a Limit operator after a [[SortExec]] operator, + * or having a [[ProjectExec]] operator between them. * This could have been named TopK, but Spark's top operator does the opposite in ordering * so we name it TakeOrdered to avoid confusion. */ -case class TakeOrderedAndProject( +case class TakeOrderedAndProjectExec( limit: Int, sortOrder: Seq[SortOrder], projectList: Option[Seq[NamedExpression]], - child: SparkPlan) extends UnaryNode { + child: SparkPlan) extends UnaryExecNode { override def output: Seq[Attribute] = { projectList.map(_.map(_.toAttribute)).getOrElse(child.output) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 7c8bc7fed8313cb6f329ac7da32d9db3770d8532..56a39069511d737b910ff03e65713338a9175a7d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.types.{DataType, ObjectType} case class DeserializeToObject( deserializer: Expression, outputObjAttr: Attribute, - child: SparkPlan) extends UnaryNode with CodegenSupport { + child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = outputObjAttr :: Nil override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) @@ -67,9 +67,9 @@ case class DeserializeToObject( * Takes the input object from child and turns in into unsafe row using the given serializer * expression. The output of its child must be a single-field row containing the input object. */ -case class SerializeFromObject( +case class SerializeFromObjectExec( serializer: Seq[NamedExpression], - child: SparkPlan) extends UnaryNode with CodegenSupport { + child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = serializer.map(_.toAttribute) @@ -136,10 +136,11 @@ trait ObjectOperator extends SparkPlan { * Applies the given function to input object iterator. * The output of its child must be a single-field row containing the input object. */ -case class MapPartitions( +case class MapPartitionsExec( func: Iterator[Any] => Iterator[Any], outputObjAttr: Attribute, - child: SparkPlan) extends UnaryNode with ObjectOperator { + child: SparkPlan) + extends UnaryExecNode with ObjectOperator { override def output: Seq[Attribute] = outputObjAttr :: Nil override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) @@ -157,13 +158,14 @@ case class MapPartitions( * Applies the given function to each input object. * The output of its child must be a single-field row containing the input object. * - * This operator is kind of a safe version of [[Project]], as it's output is custom object, we need - * to use safe row to contain it. + * This operator is kind of a safe version of [[ProjectExec]], as it's output is custom object, + * we need to use safe row to contain it. */ -case class MapElements( +case class MapElementsExec( func: AnyRef, outputObjAttr: Attribute, - child: SparkPlan) extends UnaryNode with ObjectOperator with CodegenSupport { + child: SparkPlan) + extends UnaryExecNode with ObjectOperator with CodegenSupport { override def output: Seq[Attribute] = outputObjAttr :: Nil override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) @@ -211,11 +213,11 @@ case class MapElements( /** * Applies the given function to each input row, appending the encoded result at the end of the row. */ -case class AppendColumns( +case class AppendColumnsExec( func: Any => Any, deserializer: Expression, serializer: Seq[NamedExpression], - child: SparkPlan) extends UnaryNode with ObjectOperator { + child: SparkPlan) extends UnaryExecNode with ObjectOperator { override def output: Seq[Attribute] = child.output ++ serializer.map(_.toAttribute) @@ -236,13 +238,14 @@ case class AppendColumns( } /** - * An optimized version of [[AppendColumns]], that can be executed on deserialized object directly. + * An optimized version of [[AppendColumnsExec]], that can be executed + * on deserialized object directly. */ -case class AppendColumnsWithObject( +case class AppendColumnsWithObjectExec( func: Any => Any, inputSerializer: Seq[NamedExpression], newColumnsSerializer: Seq[NamedExpression], - child: SparkPlan) extends UnaryNode with ObjectOperator { + child: SparkPlan) extends UnaryExecNode with ObjectOperator { override def output: Seq[Attribute] = (inputSerializer ++ newColumnsSerializer).map(_.toAttribute) @@ -269,14 +272,14 @@ case class AppendColumnsWithObject( * Groups the input rows together and calls the function with each group and an iterator containing * all elements in the group. The result of this function is flattened before being output. */ -case class MapGroups( +case class MapGroupsExec( func: (Any, Iterator[Any]) => TraversableOnce[Any], keyDeserializer: Expression, valueDeserializer: Expression, groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], outputObjAttr: Attribute, - child: SparkPlan) extends UnaryNode with ObjectOperator { + child: SparkPlan) extends UnaryExecNode with ObjectOperator { override def output: Seq[Attribute] = outputObjAttr :: Nil override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) @@ -310,7 +313,7 @@ case class MapGroups( * iterators containing all elements in the group from left and right side. * The result of this function is flattened before being output. */ -case class CoGroup( +case class CoGroupExec( func: (Any, Iterator[Any], Iterator[Any]) => TraversableOnce[Any], keyDeserializer: Expression, leftDeserializer: Expression, @@ -321,7 +324,7 @@ case class CoGroup( rightAttr: Seq[Attribute], outputObjAttr: Attribute, left: SparkPlan, - right: SparkPlan) extends BinaryNode with ObjectOperator { + right: SparkPlan) extends BinaryExecNode with ObjectOperator { override def output: Seq[Attribute] = outputObjAttr :: Nil override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala index c49f173ad6dff618df915a4d6b28c9b5e2a8ef1c..061d7c7f79de86bd3228f511354ff91db341e83a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.types.{DataType, StructField, StructType} * we drain the queue to find the original input row. Note that if the Python process is way too * slow, this could lead to the queue growing unbounded and eventually run out of memory. */ -case class BatchPythonEvaluation(udfs: Seq[PythonUDF], output: Seq[Attribute], child: SparkPlan) +case class BatchEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], child: SparkPlan) extends SparkPlan { def children: Seq[SparkPlan] = child :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala index d72b3d347d0f6dde556ffc7f20ac75fcda19684a..ab192360e1c1f4a96db5e0a45886dc063397e68a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala @@ -79,7 +79,7 @@ private[spark] object ExtractPythonUDFs extends Rule[SparkPlan] { val resultAttrs = udfs.zipWithIndex.map { case (u, i) => AttributeReference(s"pythonUDF$i", u.dataType)() } - val evaluation = BatchPythonEvaluation(validUdfs, child.output ++ resultAttrs, child) + val evaluation = BatchEvalPythonExec(validUdfs, child.output ++ resultAttrs, child) attributeMap ++= validUdfs.zip(resultAttrs) evaluation } else { @@ -105,7 +105,7 @@ private[spark] object ExtractPythonUDFs extends Rule[SparkPlan] { val newPlan = extract(rewritten) if (newPlan.output != plan.output) { // Trim away the new UDF value if it was only used for filtering or something. - execution.Project(plan.output, newPlan) + execution.ProjectExec(plan.output, newPlan) } else { newPlan } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 81244ed874498a8ea0c620d44e3b3b20021a9760..a1a1108447366e2eee093ff5e452268b30f82822 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -18,11 +18,10 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.analysis.{OutputMode, UnsupportedOperationChecker} +import org.apache.spark.sql.catalyst.analysis.OutputMode import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SparkPlanner, UnaryNode} -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SparkPlanner, UnaryExecNode} /** * A variant of [[QueryExecution]] that allows the execution of the given [[LogicalPlan]] @@ -54,17 +53,17 @@ class IncrementalExecution( /** Locates save/restore pairs surrounding aggregation. */ val state = new Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = plan transform { - case StateStoreSave(keys, None, - UnaryNode(agg, - StateStoreRestore(keys2, None, child))) => + case StateStoreSaveExec(keys, None, + UnaryExecNode(agg, + StateStoreRestoreExec(keys2, None, child))) => val stateId = OperatorStateId(checkpointLocation, operatorId, currentBatchId - 1) operatorId += 1 - StateStoreSave( + StateStoreSaveExec( keys, Some(stateId), agg.withNewChildren( - StateStoreRestore( + StateStoreRestoreExec( keys, Some(stateId), child) :: Nil)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala index 595774761cffeefbb2a47cf3db158ffc3786a746..de4305f5642547d04e3f5ccb5bb0827f45b8207c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala @@ -50,10 +50,11 @@ trait StatefulOperator extends SparkPlan { * For each input tuple, the key is calculated and the value from the [[StateStore]] is added * to the stream (in addition to the input tuple) if present. */ -case class StateStoreRestore( +case class StateStoreRestoreExec( keyExpressions: Seq[Attribute], stateId: Option[OperatorStateId], - child: SparkPlan) extends execution.UnaryNode with StatefulOperator { + child: SparkPlan) + extends execution.UnaryExecNode with StatefulOperator { override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsWithStateStore( @@ -78,10 +79,11 @@ case class StateStoreRestore( /** * For each input tuple, the key is calculated and the tuple is `put` into the [[StateStore]]. */ -case class StateStoreSave( +case class StateStoreSaveExec( keyExpressions: Seq[Attribute], stateId: Option[OperatorStateId], - child: SparkPlan) extends execution.UnaryNode with StatefulOperator { + child: SparkPlan) + extends execution.UnaryExecNode with StatefulOperator { override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsWithStateStore( 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 c023cc573c672123f4728cac1fef7299b4a8ae8d..1341e4548335edb7828704036e6cffedc3153437 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 @@ -40,7 +40,7 @@ case class ScalarSubquery( override def withNewPlan(plan: LogicalPlan): SubqueryExpression = { throw new UnsupportedOperationException } - override def plan: SparkPlan = Subquery(simpleString, executedPlan) + override def plan: SparkPlan = SubqueryExec(simpleString, executedPlan) override def dataType: DataType = executedPlan.schema.fields.head.dataType override def children: Seq[Expression] = Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index c6fcb6956c2740bee1d153a997536a646e91260f..1959f1e3680a0464f769b1b71cc90ee253c55767 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.sql.execution.{SparkPlanInfo, WholeStageCodegen} +import org.apache.spark.sql.execution.{SparkPlanInfo, WholeStageCodegenExec} import org.apache.spark.sql.execution.metric.SQLMetrics /** @@ -178,7 +178,7 @@ private[ui] class SparkPlanGraphCluster( extends SparkPlanGraphNode(id, name, desc, Map.empty, metrics) { override def makeDotNode(metricsValue: Map[Long, String]): String = { - val duration = metrics.filter(_.name.startsWith(WholeStageCodegen.PIPELINE_DURATION_METRIC)) + val duration = metrics.filter(_.name.startsWith(WholeStageCodegenExec.PIPELINE_DURATION_METRIC)) val labelStr = if (duration.nonEmpty) { require(duration.length == 1) val id = duration(0).accumulatorId diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 82b79c791db402b4904fd28957b29f875bf57fe5..4aea21e52a685c8f2721ef2861a1d519277c95c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -23,7 +23,7 @@ import scala.language.postfixOps import org.scalatest.concurrent.Eventually._ import org.apache.spark.Accumulators -import org.apache.spark.sql.execution.PhysicalRDD +import org.apache.spark.sql.execution.RDDScanExec import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.functions._ @@ -38,7 +38,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext def rddIdOf(tableName: String): Int = { val plan = sqlContext.table(tableName).queryExecution.sparkPlan plan.collect { - case InMemoryColumnarTableScan(_, _, relation) => + case InMemoryTableScanExec(_, _, relation) => relation.cachedColumnBuffers.id case _ => fail(s"Table $tableName is not cached\n" + plan) @@ -167,7 +167,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext sqlContext.cacheTable("testData") assertResult(0, "Double InMemoryRelations found, cacheTable() is not idempotent") { sqlContext.table("testData").queryExecution.withCachedData.collect { - case r @ InMemoryRelation(_, _, _, _, _: InMemoryColumnarTableScan, _) => r + case r @ InMemoryRelation(_, _, _, _, _: InMemoryTableScanExec, _) => r }.size } @@ -351,8 +351,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext |abc a join abc b on a.key=b.key |join abc c on a.key=c.key""".stripMargin).queryExecution.sparkPlan - assert(sparkPlan.collect { case e: InMemoryColumnarTableScan => e }.size === 3) - assert(sparkPlan.collect { case e: PhysicalRDD => e }.size === 0) + assert(sparkPlan.collect { case e: InMemoryTableScanExec => e }.size === 3) + assert(sparkPlan.collect { case e: RDDScanExec => e }.size === 0) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 351b03b38bad1e9d0b4e6f43d2744cf9bb23859b..19fe29a202a64b079e84651460d51a4e2fa17589 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import org.scalatest.Matchers._ import org.apache.spark.sql.catalyst.expressions.NamedExpression -import org.apache.spark.sql.execution.Project +import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -631,7 +631,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { def checkNumProjects(df: DataFrame, expectedNumProjects: Int): Unit = { val projects = df.queryExecution.sparkPlan.collect { - case tungstenProject: Project => tungstenProject + case tungstenProject: ProjectExec => tungstenProject } assert(projects.size === expectedNumProjects) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 067a62d011ec493d11f89f914dc8efa6d609ca0b..0414fa1c91efa4049f7a3bf3cd2aacae01f76195 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.plans.logical.Join -import org.apache.spark.sql.execution.joins.BroadcastHashJoin +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext @@ -142,11 +142,11 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { // equijoin - should be converted into broadcast join val plan1 = df1.join(broadcast(df2), "key").queryExecution.sparkPlan - assert(plan1.collect { case p: BroadcastHashJoin => p }.size === 1) + assert(plan1.collect { case p: BroadcastHashJoinExec => p }.size === 1) // no join key -- should not be a broadcast join val plan2 = df1.join(broadcast(df2)).queryExecution.sparkPlan - assert(plan2.collect { case p: BroadcastHashJoin => p }.size === 0) + assert(plan2.collect { case p: BroadcastHashJoinExec => p }.size === 0) // planner should not crash without a join broadcast(df1).queryExecution.sparkPlan diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index e953a6e8ef0c2b80369d4c0db8b01f3ef20cf584..4c18784126751f769817cf6a14650cbac66bc86d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union} import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.aggregate.TungstenAggregate -import org.apache.spark.sql.execution.exchange.{BroadcastExchange, ReusedExchange, ShuffleExchange} +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchange} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext} @@ -1355,16 +1355,18 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer(join, df) assert( join.queryExecution.executedPlan.collect { case e: ShuffleExchange => true }.size === 1) - assert(join.queryExecution.executedPlan.collect { case e: ReusedExchange => true }.size === 1) + assert( + join.queryExecution.executedPlan.collect { case e: ReusedExchangeExec => true }.size === 1) val broadcasted = broadcast(join) val join2 = join.join(broadcasted, "id").join(broadcasted, "id") checkAnswer(join2, df) assert( join2.queryExecution.executedPlan.collect { case e: ShuffleExchange => true }.size === 1) assert( - join2.queryExecution.executedPlan.collect { case e: BroadcastExchange => true }.size === 1) + join2.queryExecution.executedPlan + .collect { case e: BroadcastExchangeExec => true }.size === 1) assert( - join2.queryExecution.executedPlan.collect { case e: ReusedExchange => true }.size === 4) + join2.queryExecution.executedPlan.collect { case e: ReusedExchangeExec => true }.size === 4) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index a87a41c12664fe894bb5fed371d267268801e2d6..9e5a41d57cc7e4e2e6f825788af5228013e096a9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -46,11 +46,11 @@ class JoinSuite extends QueryTest with SharedSQLContext { val df = sql(sqlString) val physical = df.queryExecution.sparkPlan val operators = physical.collect { - case j: BroadcastHashJoin => j - case j: ShuffledHashJoin => j - case j: CartesianProduct => j - case j: BroadcastNestedLoopJoin => j - case j: SortMergeJoin => j + case j: BroadcastHashJoinExec => j + case j: ShuffledHashJoinExec => j + case j: CartesianProductExec => j + case j: BroadcastNestedLoopJoinExec => j + case j: SortMergeJoinExec => j } assert(operators.size === 1) @@ -64,39 +64,43 @@ class JoinSuite extends QueryTest with SharedSQLContext { withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "0") { Seq( - ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[BroadcastNestedLoopJoin]), - ("SELECT * FROM testData JOIN testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), - ("SELECT * FROM testData LEFT JOIN testData2", classOf[BroadcastNestedLoopJoin]), - ("SELECT * FROM testData RIGHT JOIN testData2", classOf[BroadcastNestedLoopJoin]), - ("SELECT * FROM testData FULL OUTER JOIN testData2", classOf[BroadcastNestedLoopJoin]), + ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", + classOf[ShuffledHashJoinExec]), + ("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData JOIN testData2", classOf[CartesianProductExec]), + ("SELECT * FROM testData JOIN testData2 WHERE key = 2", classOf[CartesianProductExec]), + ("SELECT * FROM testData LEFT JOIN testData2", classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData RIGHT JOIN testData2", classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData FULL OUTER JOIN testData2", classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData LEFT JOIN testData2 WHERE key = 2", - classOf[BroadcastNestedLoopJoin]), - ("SELECT * FROM testData RIGHT JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), + classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData RIGHT JOIN testData2 WHERE key = 2", + classOf[CartesianProductExec]), ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key = 2", - classOf[BroadcastNestedLoopJoin]), - ("SELECT * FROM testData JOIN testData2 WHERE key > a", classOf[CartesianProduct]), + classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData JOIN testData2 WHERE key > a", classOf[CartesianProductExec]), ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key > a", - classOf[CartesianProduct]), - ("SELECT * FROM testData JOIN testData2 ON key = a", classOf[SortMergeJoin]), - ("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", classOf[SortMergeJoin]), - ("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", classOf[SortMergeJoin]), - ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", classOf[SortMergeJoin]), + classOf[CartesianProductExec]), + ("SELECT * FROM testData JOIN testData2 ON key = a", classOf[SortMergeJoinExec]), + ("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", + classOf[SortMergeJoinExec]), + ("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", + classOf[SortMergeJoinExec]), + ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", classOf[SortMergeJoinExec]), ("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2", - classOf[SortMergeJoin]), + classOf[SortMergeJoinExec]), ("SELECT * FROM testData right join testData2 ON key = a and key = 2", - classOf[SortMergeJoin]), + classOf[SortMergeJoinExec]), ("SELECT * FROM testData full outer join testData2 ON key = a", - classOf[SortMergeJoin]), + classOf[SortMergeJoinExec]), ("SELECT * FROM testData left JOIN testData2 ON (key * a != key + a)", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData right JOIN testData2 ON (key * a != key + a)", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData full JOIN testData2 ON (key * a != key + a)", - classOf[BroadcastNestedLoopJoin]), - ("SELECT * FROM testData ANTI JOIN testData2 ON key = a", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData LEFT ANTI JOIN testData2", classOf[BroadcastNestedLoopJoin]) + classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData ANTI JOIN testData2 ON key = a", classOf[ShuffledHashJoinExec]), + ("SELECT * FROM testData LEFT ANTI JOIN testData2", classOf[BroadcastNestedLoopJoinExec]) ).foreach(assertJoin) } } @@ -112,11 +116,11 @@ class JoinSuite extends QueryTest with SharedSQLContext { sql("CACHE TABLE testData") Seq( ("SELECT * FROM testData join testData2 ON key = a", - classOf[BroadcastHashJoin]), + classOf[BroadcastHashJoinExec]), ("SELECT * FROM testData join testData2 ON key = a and key = 2", - classOf[BroadcastHashJoin]), + classOf[BroadcastHashJoinExec]), ("SELECT * FROM testData join testData2 ON key = a where key = 2", - classOf[BroadcastHashJoin]) + classOf[BroadcastHashJoinExec]) ).foreach(assertJoin) sql("UNCACHE TABLE testData") } @@ -127,11 +131,11 @@ class JoinSuite extends QueryTest with SharedSQLContext { sql("CACHE TABLE testData2") Seq( ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", - classOf[BroadcastHashJoin]), + classOf[BroadcastHashJoinExec]), ("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2", - classOf[BroadcastHashJoin]), + classOf[BroadcastHashJoinExec]), ("SELECT * FROM testData right join testData2 ON key = a and key = 2", - classOf[BroadcastHashJoin]) + classOf[BroadcastHashJoinExec]) ).foreach(assertJoin) sql("UNCACHE TABLE testData") } @@ -428,15 +432,18 @@ class JoinSuite extends QueryTest with SharedSQLContext { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000000000") { Seq( - ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[BroadcastHashJoin]), - ("SELECT * FROM testData ANT JOIN testData2 ON key = a", classOf[BroadcastHashJoin]) + ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", + classOf[BroadcastHashJoinExec]), + ("SELECT * FROM testData ANT JOIN testData2 ON key = a", classOf[BroadcastHashJoinExec]) ).foreach(assertJoin) } withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { Seq( - ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData LEFT ANTI JOIN testData2 ON key = a", classOf[ShuffledHashJoin]) + ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", + classOf[ShuffledHashJoinExec]), + ("SELECT * FROM testData LEFT ANTI JOIN testData2 ON key = a", + classOf[ShuffledHashJoinExec]) ).foreach(assertJoin) } @@ -460,35 +467,35 @@ class JoinSuite extends QueryTest with SharedSQLContext { Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", - classOf[ShuffledHashJoin]), + classOf[ShuffledHashJoinExec]), ("SELECT * FROM testData LEFT SEMI JOIN testData2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData JOIN testData2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData JOIN testData2 WHERE key = 2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData LEFT JOIN testData2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData RIGHT JOIN testData2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData FULL OUTER JOIN testData2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData LEFT JOIN testData2 WHERE key = 2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData RIGHT JOIN testData2 WHERE key = 2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key = 2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData JOIN testData2 WHERE key > a", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key > a", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData left JOIN testData2 WHERE (key * a != key + a)", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData right JOIN testData2 WHERE (key * a != key + a)", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData full JOIN testData2 WHERE (key * a != key + a)", - classOf[BroadcastNestedLoopJoin]) + classOf[BroadcastNestedLoopJoinExec]) ).foreach(assertJoin) checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 9e640493cf7327f7d9d002f6c7ba20ba362da3d9..84f0c0083b3bf16e328fd5dc241e9eac584f9c7e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.SortOrder import org.apache.spark.sql.catalyst.plans.logical.Aggregate import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.aggregate -import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, CartesianProduct, SortMergeJoin} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} @@ -866,12 +866,12 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-11111 null-safe join should not use cartesian product") { val df = sql("select count(*) from testData a join testData b on (a.key <=> b.key)") val cp = df.queryExecution.sparkPlan.collect { - case cp: CartesianProduct => cp + case cp: CartesianProductExec => cp } assert(cp.isEmpty, "should not use CartesianProduct for null-safe join") val smj = df.queryExecution.sparkPlan.collect { - case smj: SortMergeJoin => smj - case j: BroadcastHashJoin => j + case smj: SortMergeJoinExec => smj + case j: BroadcastHashJoinExec => j } assert(smj.size > 0, "should use SortMergeJoin or BroadcastHashJoin") checkAnswer(df, Row(100) :: Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala index 17f2343cf971ec18e80f8ece37d241cf8b64b5a0..ba16810ceeb50e1f4f9026986263385da329a0f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.{Alias, Literal} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, IdentityBroadcastMode, SinglePartition} -import org.apache.spark.sql.execution.exchange.{BroadcastExchange, ReusedExchange, ShuffleExchange} +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchange} import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode import org.apache.spark.sql.test.SharedSQLContext @@ -55,13 +55,13 @@ class ExchangeSuite extends SparkPlanTest with SharedSQLContext { val output = plan.output assert(plan sameResult plan) - val exchange1 = BroadcastExchange(IdentityBroadcastMode, plan) + val exchange1 = BroadcastExchangeExec(IdentityBroadcastMode, plan) val hashMode = HashedRelationBroadcastMode(output) - val exchange2 = BroadcastExchange(hashMode, plan) + val exchange2 = BroadcastExchangeExec(hashMode, plan) val hashMode2 = HashedRelationBroadcastMode(Alias(output.head, "id2")() :: Nil) - val exchange3 = BroadcastExchange(hashMode2, plan) - val exchange4 = ReusedExchange(output, exchange3) + val exchange3 = BroadcastExchangeExec(hashMode2, plan) + val exchange4 = ReusedExchangeExec(output, exchange3) assert(exchange1 sameResult exchange1) assert(exchange2 sameResult exchange2) @@ -87,7 +87,7 @@ class ExchangeSuite extends SparkPlanTest with SharedSQLContext { val exchange3 = ShuffleExchange(part2, plan) val part3 = HashPartitioning(output ++ output, 2) val exchange4 = ShuffleExchange(part3, plan) - val exchange5 = ReusedExchange(output, exchange4) + val exchange5 = ReusedExchangeExec(output, exchange4) assert(exchange1 sameResult exchange1) assert(exchange2 sameResult exchange2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index bdbcf842ca47d58e8ff65b623bcb48f85b0b570d..3b2911d0566523a07be73282a05f642791d859ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -25,8 +25,8 @@ import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Repartition} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.columnar.InMemoryRelation -import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchange, ReuseExchange, ShuffleExchange} -import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, SortMergeJoin} +import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchangeExec, ReuseExchange, ShuffleExchange} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -86,8 +86,8 @@ class PlannerSuite extends SharedSQLContext { |FROM testData2 l JOIN (SELECT * FROM testLimit LIMIT 1) r ON (l.a = r.key) """.stripMargin).queryExecution.sparkPlan - val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } - val sortMergeJoins = planned.collect { case join: SortMergeJoin => join } + val broadcastHashJoins = planned.collect { case join: BroadcastHashJoinExec => join } + val sortMergeJoins = planned.collect { case join: SortMergeJoinExec => join } assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") assert(sortMergeJoins.isEmpty, "Should not use sort merge join") @@ -139,8 +139,8 @@ class PlannerSuite extends SharedSQLContext { val b = sqlContext.table("tiny").as("b") val planned = a.join(b, $"a.key" === $"b.key").queryExecution.sparkPlan - val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } - val sortMergeJoins = planned.collect { case join: SortMergeJoin => join } + val broadcastHashJoins = planned.collect { case join: BroadcastHashJoinExec => join } + val sortMergeJoins = planned.collect { case join: SortMergeJoinExec => join } assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") assert(sortMergeJoins.isEmpty, "Should not use shuffled hash join") @@ -167,34 +167,34 @@ class PlannerSuite extends SharedSQLContext { test("efficient terminal limit -> sort should use TakeOrderedAndProject") { val query = testData.select('key, 'value).sort('key).limit(2) val planned = query.queryExecution.executedPlan - assert(planned.isInstanceOf[execution.TakeOrderedAndProject]) + assert(planned.isInstanceOf[execution.TakeOrderedAndProjectExec]) assert(planned.output === testData.select('key, 'value).logicalPlan.output) } test("terminal limit -> project -> sort should use TakeOrderedAndProject") { val query = testData.select('key, 'value).sort('key).select('value, 'key).limit(2) val planned = query.queryExecution.executedPlan - assert(planned.isInstanceOf[execution.TakeOrderedAndProject]) + assert(planned.isInstanceOf[execution.TakeOrderedAndProjectExec]) assert(planned.output === testData.select('value, 'key).logicalPlan.output) } test("terminal limits that are not handled by TakeOrderedAndProject should use CollectLimit") { val query = testData.select('value).limit(2) val planned = query.queryExecution.sparkPlan - assert(planned.isInstanceOf[CollectLimit]) + assert(planned.isInstanceOf[CollectLimitExec]) assert(planned.output === testData.select('value).logicalPlan.output) } test("TakeOrderedAndProject can appear in the middle of plans") { val query = testData.select('key, 'value).sort('key).limit(2).filter('key === 3) val planned = query.queryExecution.executedPlan - assert(planned.find(_.isInstanceOf[TakeOrderedAndProject]).isDefined) + assert(planned.find(_.isInstanceOf[TakeOrderedAndProjectExec]).isDefined) } test("CollectLimit can appear in the middle of a plan when caching is used") { val query = testData.select('key, 'value).limit(2).cache() val planned = query.queryExecution.optimizedPlan.asInstanceOf[InMemoryRelation] - assert(planned.child.isInstanceOf[CollectLimit]) + assert(planned.child.isInstanceOf[CollectLimitExec]) } test("PartitioningCollection") { @@ -394,7 +394,7 @@ class PlannerSuite extends SharedSQLContext { ) val outputPlan = EnsureRequirements(sqlContext.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case s: Sort => true }.isEmpty) { + if (outputPlan.collect { case s: SortExec => true }.isEmpty) { fail(s"Sort should have been added:\n$outputPlan") } } @@ -410,7 +410,7 @@ class PlannerSuite extends SharedSQLContext { ) val outputPlan = EnsureRequirements(sqlContext.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case s: Sort => true }.nonEmpty) { + if (outputPlan.collect { case s: SortExec => true }.nonEmpty) { fail(s"No sorts should have been added:\n$outputPlan") } } @@ -427,7 +427,7 @@ class PlannerSuite extends SharedSQLContext { ) val outputPlan = EnsureRequirements(sqlContext.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case s: Sort => true }.isEmpty) { + if (outputPlan.collect { case s: SortExec => true }.isEmpty) { fail(s"Sort should have been added:\n$outputPlan") } } @@ -485,7 +485,7 @@ class PlannerSuite extends SharedSQLContext { requiredChildOrdering = Seq(Seq.empty)), None) - val inputPlan = SortMergeJoin( + val inputPlan = SortMergeJoinExec( Literal(1) :: Nil, Literal(1) :: Nil, Inner, @@ -494,7 +494,7 @@ class PlannerSuite extends SharedSQLContext { shuffle) val outputPlan = ReuseExchange(sqlContext.sessionState.conf).apply(inputPlan) - if (outputPlan.collect { case e: ReusedExchange => true }.size != 1) { + if (outputPlan.collect { case e: ReusedExchangeExec => true }.size != 1) { fail(s"Should re-use the shuffle:\n$outputPlan") } if (outputPlan.collect { case e: ShuffleExchange => true }.size != 1) { @@ -502,7 +502,7 @@ class PlannerSuite extends SharedSQLContext { } // nested exchanges - val inputPlan2 = SortMergeJoin( + val inputPlan2 = SortMergeJoinExec( Literal(1) :: Nil, Literal(1) :: Nil, Inner, @@ -511,7 +511,7 @@ class PlannerSuite extends SharedSQLContext { ShuffleExchange(finalPartitioning, inputPlan)) val outputPlan2 = ReuseExchange(sqlContext.sessionState.conf).apply(inputPlan2) - if (outputPlan2.collect { case e: ReusedExchange => true }.size != 2) { + if (outputPlan2.collect { case e: ReusedExchangeExec => true }.size != 2) { fail(s"Should re-use the two shuffles:\n$outputPlan2") } if (outputPlan2.collect { case e: ShuffleExchange => true }.size != 2) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala index 2963a856d15cf4cd5a8c558d064dc28753c15b43..a19ea51af7c017507c2ab4b117c8ca293925f7e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala @@ -34,7 +34,7 @@ case class ReferenceSort( sortOrder: Seq[SortOrder], global: Boolean, child: SparkPlan) - extends UnaryNode { + extends UnaryExecNode { override def requiredChildDistribution: Seq[Distribution] = if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala index 778477660e1690651a0b646c308640e1ace9c6c4..ebeb39b690e6944ed3bcfd73a75db0129840a6b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala @@ -43,13 +43,13 @@ class SortSuite extends SparkPlanTest with SharedSQLContext { checkAnswer( input.toDF("a", "b", "c"), - (child: SparkPlan) => Sort('a.asc :: 'b.asc :: Nil, global = true, child = child), + (child: SparkPlan) => SortExec('a.asc :: 'b.asc :: Nil, global = true, child = child), input.sortBy(t => (t._1, t._2)).map(Row.fromTuple), sortAnswers = false) checkAnswer( input.toDF("a", "b", "c"), - (child: SparkPlan) => Sort('b.asc :: 'a.asc :: Nil, global = true, child = child), + (child: SparkPlan) => SortExec('b.asc :: 'a.asc :: Nil, global = true, child = child), input.sortBy(t => (t._2, t._1)).map(Row.fromTuple), sortAnswers = false) } @@ -57,8 +57,10 @@ class SortSuite extends SparkPlanTest with SharedSQLContext { test("sort followed by limit") { checkThatPlansAgree( (1 to 100).map(v => Tuple1(v)).toDF("a"), - (child: SparkPlan) => GlobalLimit(10, Sort('a.asc :: Nil, global = true, child = child)), - (child: SparkPlan) => GlobalLimit(10, ReferenceSort('a.asc :: Nil, global = true, child)), + (child: SparkPlan) => + GlobalLimitExec(10, SortExec('a.asc :: Nil, global = true, child = child)), + (child: SparkPlan) => + GlobalLimitExec(10, ReferenceSort('a.asc :: Nil, global = true, child)), sortAnswers = false ) } @@ -68,7 +70,7 @@ class SortSuite extends SparkPlanTest with SharedSQLContext { val stringLength = 1024 * 1024 * 2 checkThatPlansAgree( Seq(Tuple1("a" * stringLength), Tuple1("b" * stringLength)).toDF("a").repartition(1), - Sort(sortOrder, global = true, _: SparkPlan, testSpillFrequency = 1), + SortExec(sortOrder, global = true, _: SparkPlan, testSpillFrequency = 1), ReferenceSort(sortOrder, global = true, _: SparkPlan), sortAnswers = false ) @@ -78,7 +80,7 @@ class SortSuite extends SparkPlanTest with SharedSQLContext { AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "unsafe external sort") { checkThatPlansAgree( (1 to 100).map(v => Tuple1(v)).toDF("a"), - (child: SparkPlan) => Sort('a.asc :: Nil, global = true, child = child), + (child: SparkPlan) => SortExec('a.asc :: Nil, global = true, child = child), (child: SparkPlan) => ReferenceSort('a.asc :: Nil, global = true, child), sortAnswers = false) } @@ -99,7 +101,7 @@ class SortSuite extends SparkPlanTest with SharedSQLContext { ) checkThatPlansAgree( inputDf, - p => Sort(sortOrder, global = true, p: SparkPlan, testSpillFrequency = 23), + p => SortExec(sortOrder, global = true, p: SparkPlan, testSpillFrequency = 23), ReferenceSort(sortOrder, global = true, _: SparkPlan), sortAnswers = false ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala index a4c6d072f33a8e5f376f43b2326887238549d97a..fba04d0cb265375840e2f554f8ddea0e013ead2a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala @@ -49,7 +49,7 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { * Adds a no-op filter to the child plan in order to prevent executeCollect() from being * called directly on the child plan. */ - private def noOpFilter(plan: SparkPlan): SparkPlan = Filter(Literal(true), plan) + private def noOpFilter(plan: SparkPlan): SparkPlan = FilterExec(Literal(true), plan) val limit = 250 val sortOrder = 'a.desc :: 'b.desc :: Nil @@ -59,11 +59,11 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { checkThatPlansAgree( generateRandomInputData(), input => - noOpFilter(TakeOrderedAndProject(limit, sortOrder, None, input)), + noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, None, input)), input => - GlobalLimit(limit, - LocalLimit(limit, - Sort(sortOrder, true, input))), + GlobalLimitExec(limit, + LocalLimitExec(limit, + SortExec(sortOrder, true, input))), sortAnswers = false) } } @@ -73,12 +73,13 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { checkThatPlansAgree( generateRandomInputData(), input => - noOpFilter(TakeOrderedAndProject(limit, sortOrder, Some(Seq(input.output.last)), input)), + noOpFilter( + TakeOrderedAndProjectExec(limit, sortOrder, Some(Seq(input.output.last)), input)), input => - GlobalLimit(limit, - LocalLimit(limit, - Project(Seq(input.output.last), - Sort(sortOrder, true, input)))), + GlobalLimitExec(limit, + LocalLimitExec(limit, + ProjectExec(Seq(input.output.last), + SortExec(sortOrder, true, input)))), sortAnswers = false) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index d7cf1dc6aadb4c2eb7fcb84092d4d1d12284775b..233104ae84fdf5d71f36381f27254c1a144edd0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.Row import org.apache.spark.sql.execution.aggregate.TungstenAggregate -import org.apache.spark.sql.execution.joins.BroadcastHashJoin +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.expressions.scala.typed import org.apache.spark.sql.functions.{avg, broadcast, col, max} import org.apache.spark.sql.test.SharedSQLContext @@ -30,7 +30,7 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { test("range/filter should be combined") { val df = sqlContext.range(10).filter("id = 1").selectExpr("id + 1") val plan = df.queryExecution.executedPlan - assert(plan.find(_.isInstanceOf[WholeStageCodegen]).isDefined) + assert(plan.find(_.isInstanceOf[WholeStageCodegenExec]).isDefined) assert(df.collect() === Array(Row(2))) } @@ -38,8 +38,8 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val df = sqlContext.range(10).groupBy().agg(max(col("id")), avg(col("id"))) val plan = df.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[TungstenAggregate]).isDefined) assert(df.collect() === Array(Row(9, 4.5))) } @@ -47,8 +47,8 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val df = sqlContext.range(3).groupBy("id").count().orderBy("id") val plan = df.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[TungstenAggregate]).isDefined) assert(df.collect() === Array(Row(0, 1), Row(1, 1), Row(2, 1))) } @@ -58,8 +58,8 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val smallDF = sqlContext.createDataFrame(rdd, schema) val df = sqlContext.range(10).join(broadcast(smallDF), col("k") === col("id")) assert(df.queryExecution.executedPlan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[BroadcastHashJoin]).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[BroadcastHashJoinExec]).isDefined) assert(df.collect() === Array(Row(1, 1, "1"), Row(1, 1, "1"), Row(2, 2, "2"))) } @@ -67,8 +67,8 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val df = sqlContext.range(3, 0, -1).toDF().sort(col("id")) val plan = df.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[Sort]).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec]).isDefined) assert(df.collect() === Array(Row(1), Row(2), Row(3))) } @@ -78,8 +78,8 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val ds = sqlContext.range(10).map(_.toString) val plan = ds.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[SerializeFromObject]).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SerializeFromObjectExec]).isDefined) assert(ds.collect() === 0.until(10).map(_.toString).toArray) } @@ -87,8 +87,8 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val ds = sqlContext.range(10).filter(_ % 2 == 0) val plan = ds.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[Filter]).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[FilterExec]).isDefined) assert(ds.collect() === Array(0, 2, 4, 6, 8)) } @@ -96,8 +96,8 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val ds = sqlContext.range(10).filter(_ % 2 == 0).filter(_ % 3 == 0) val plan = ds.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[SerializeFromObject]).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SerializeFromObjectExec]).isDefined) assert(ds.collect() === Array(0, 6)) } @@ -109,8 +109,8 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val plan = ds.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[TungstenAggregate]).isDefined) assert(ds.collect() === Array(("a", 10.0), ("b", 3.0), ("c", 1.0))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala index 4f185ed283ce65e5f0fb6d0b0cc787e1da388d9a..9164074a3e996b688e9988c94d5ace4e11f447f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala @@ -133,7 +133,7 @@ class PartitionBatchPruningSuite } val (readPartitions, readBatches) = df.queryExecution.sparkPlan.collect { - case in: InMemoryColumnarTableScan => (in.readPartitions.value, in.readBatches.value) + case in: InMemoryTableScanExec => (in.readPartitions.value, in.readBatches.value) }.head assert(readBatches === expectedReadBatches, s"Wrong number of read batches: $queryExecution") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 50cd03a40c6ac3803f99717ea6000b2d0bf0e62c..fb70dbd961dc76a631393e196be06e8b1d0172e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionSet, PredicateHelper} import org.apache.spark.sql.catalyst.util -import org.apache.spark.sql.execution.DataSourceScan +import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ @@ -375,7 +375,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi def getPhysicalFilters(df: DataFrame): ExpressionSet = { ExpressionSet( df.queryExecution.executedPlan.collect { - case execution.Filter(f, _) => splitConjunctivePredicates(f) + case execution.FilterExec(f, _) => splitConjunctivePredicates(f) }.flatten) } @@ -422,7 +422,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi def getFileScanRDD(df: DataFrame): FileScanRDD = { df.queryExecution.executedPlan.collect { - case scan: DataSourceScan if scan.rdd.isInstanceOf[FileScanRDD] => + case scan: DataSourceScanExec if scan.rdd.isInstanceOf[FileScanRDD] => scan.rdd.asInstanceOf[FileScanRDD] }.headOption.getOrElse { fail(s"No FileScan in query\n${df.queryExecution}") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index babe7ef70f99d3c7db2a1618f4e45ef66262c13b..b9df43d04984ee76303c435f87646d45ef43bcad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -71,15 +71,15 @@ class BroadcastJoinSuite extends QueryTest with BeforeAndAfterAll { } test("unsafe broadcast hash join updates peak execution memory") { - testBroadcastJoin[BroadcastHashJoin]("unsafe broadcast hash join", "inner") + testBroadcastJoin[BroadcastHashJoinExec]("unsafe broadcast hash join", "inner") } test("unsafe broadcast hash outer join updates peak execution memory") { - testBroadcastJoin[BroadcastHashJoin]("unsafe broadcast hash outer join", "left_outer") + testBroadcastJoin[BroadcastHashJoinExec]("unsafe broadcast hash outer join", "left_outer") } test("unsafe broadcast left semi join updates peak execution memory") { - testBroadcastJoin[BroadcastHashJoin]("unsafe broadcast left semi join", "leftsemi") + testBroadcastJoin[BroadcastHashJoinExec]("unsafe broadcast left semi join", "leftsemi") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala index 8cdfa8afd098acb03d6eb7c4fe641c490adb57be..bc838ee4da2ca09e6987812f1999a7cdc51f4902 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala @@ -83,7 +83,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(left.sqlContext.sessionState.conf).apply( - ShuffledHashJoin( + ShuffledHashJoinExec( leftKeys, rightKeys, joinType, BuildRight, boundCondition, left, right)), expectedAnswer, sortAnswers = true) @@ -96,7 +96,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(left.sqlContext.sessionState.conf).apply( - BroadcastHashJoin( + BroadcastHashJoinExec( leftKeys, rightKeys, joinType, BuildRight, boundCondition, left, right)), expectedAnswer, sortAnswers = true) @@ -108,7 +108,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(left.sqlContext.sessionState.conf).apply( - BroadcastNestedLoopJoin(left, right, BuildLeft, joinType, Some(condition))), + BroadcastNestedLoopJoinExec(left, right, BuildLeft, joinType, Some(condition))), expectedAnswer, sortAnswers = true) } @@ -118,7 +118,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(left.sqlContext.sessionState.conf).apply( - BroadcastNestedLoopJoin(left, right, BuildRight, joinType, Some(condition))), + BroadcastNestedLoopJoinExec(left, right, BuildRight, joinType, Some(condition))), expectedAnswer, sortAnswers = true) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index 3cb3ef1ffa2f4c9c60153a5b4150511fdd710f06..933f32e496f8f0d2b9a49b3d7503dc3ec935ec16 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -91,7 +91,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { leftPlan: SparkPlan, rightPlan: SparkPlan, side: BuildSide) = { - val broadcastJoin = joins.BroadcastHashJoin( + val broadcastJoin = joins.BroadcastHashJoinExec( leftKeys, rightKeys, Inner, @@ -110,9 +110,9 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { rightPlan: SparkPlan, side: BuildSide) = { val shuffledHashJoin = - joins.ShuffledHashJoin(leftKeys, rightKeys, Inner, side, None, leftPlan, rightPlan) + joins.ShuffledHashJoinExec(leftKeys, rightKeys, Inner, side, None, leftPlan, rightPlan) val filteredJoin = - boundCondition.map(Filter(_, shuffledHashJoin)).getOrElse(shuffledHashJoin) + boundCondition.map(FilterExec(_, shuffledHashJoin)).getOrElse(shuffledHashJoin) EnsureRequirements(sqlContext.sessionState.conf).apply(filteredJoin) } @@ -123,7 +123,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { leftPlan: SparkPlan, rightPlan: SparkPlan) = { val sortMergeJoin = - joins.SortMergeJoin(leftKeys, rightKeys, Inner, boundCondition, leftPlan, rightPlan) + joins.SortMergeJoinExec(leftKeys, rightKeys, Inner, boundCondition, leftPlan, rightPlan) EnsureRequirements(sqlContext.sessionState.conf).apply(sortMergeJoin) } @@ -189,7 +189,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { test(s"$testName using CartesianProduct") { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - Filter(condition(), CartesianProduct(left, right)), + FilterExec(condition(), CartesianProductExec(left, right)), expectedAnswer.map(Row.fromTuple), sortAnswers = true) } @@ -198,7 +198,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { test(s"$testName using BroadcastNestedLoopJoin build left") { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - BroadcastNestedLoopJoin(left, right, BuildLeft, Inner, Some(condition())), + BroadcastNestedLoopJoinExec(left, right, BuildLeft, Inner, Some(condition())), expectedAnswer.map(Row.fromTuple), sortAnswers = true) } @@ -207,7 +207,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { test(s"$testName using BroadcastNestedLoopJoin build right") { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - BroadcastNestedLoopJoin(left, right, BuildRight, Inner, Some(condition())), + BroadcastNestedLoopJoinExec(left, right, BuildRight, Inner, Some(condition())), expectedAnswer.map(Row.fromTuple), sortAnswers = true) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala index 4cacb20aa0791acbc5cc974b8c7b5999cde05fd1..c26cb8483eb1773a9fd1a346ce329f5d2d658308 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala @@ -83,7 +83,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { val buildSide = if (joinType == LeftOuter) BuildRight else BuildLeft checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(sqlContext.sessionState.conf).apply( - ShuffledHashJoin( + ShuffledHashJoinExec( leftKeys, rightKeys, joinType, buildSide, boundCondition, left, right)), expectedAnswer.map(Row.fromTuple), sortAnswers = true) @@ -102,7 +102,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - BroadcastHashJoin( + BroadcastHashJoinExec( leftKeys, rightKeys, joinType, buildSide, boundCondition, left, right), expectedAnswer.map(Row.fromTuple), sortAnswers = true) @@ -116,7 +116,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(sqlContext.sessionState.conf).apply( - SortMergeJoin(leftKeys, rightKeys, joinType, boundCondition, left, right)), + SortMergeJoinExec(leftKeys, rightKeys, joinType, boundCondition, left, right)), expectedAnswer.map(Row.fromTuple), sortAnswers = true) } @@ -126,7 +126,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { test(s"$testName using BroadcastNestedLoopJoin build left") { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - BroadcastNestedLoopJoin(left, right, BuildLeft, joinType, Some(condition)), + BroadcastNestedLoopJoinExec(left, right, BuildLeft, joinType, Some(condition)), expectedAnswer.map(Row.fromTuple), sortAnswers = true) } @@ -135,7 +135,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { test(s"$testName using BroadcastNestedLoopJoin build right") { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - BroadcastNestedLoopJoin(left, right, BuildRight, joinType, Some(condition)), + BroadcastNestedLoopJoinExec(left, right, BuildRight, joinType, Some(condition)), expectedAnswer.map(Row.fromTuple), sortAnswers = true) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index f66deea06589c7ebdddec101bf2821a6115ba7a2..c24abf165010e34c38b5288e60d296f58a21b702 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.{BeforeAndAfter, PrivateMethodTester} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.{DataFrame, Row} -import org.apache.spark.sql.execution.DataSourceScan +import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD @@ -208,10 +208,10 @@ class JDBCSuite extends SparkFunSuite val parentPlan = df.queryExecution.executedPlan // Check if SparkPlan Filter is removed in a physical plan and // the plan only has PhysicalRDD to scan JDBCRelation. - assert(parentPlan.isInstanceOf[org.apache.spark.sql.execution.WholeStageCodegen]) - val node = parentPlan.asInstanceOf[org.apache.spark.sql.execution.WholeStageCodegen] - assert(node.child.isInstanceOf[org.apache.spark.sql.execution.DataSourceScan]) - assert(node.child.asInstanceOf[DataSourceScan].nodeName.contains("JDBCRelation")) + assert(parentPlan.isInstanceOf[org.apache.spark.sql.execution.WholeStageCodegenExec]) + val node = parentPlan.asInstanceOf[org.apache.spark.sql.execution.WholeStageCodegenExec] + assert(node.child.isInstanceOf[org.apache.spark.sql.execution.DataSourceScanExec]) + assert(node.child.asInstanceOf[DataSourceScanExec].nodeName.contains("JDBCRelation")) df } assert(checkPushdown(sql("SELECT * FROM foobar WHERE THEID < 1")).collect().size == 0) @@ -246,9 +246,9 @@ class JDBCSuite extends SparkFunSuite val parentPlan = df.queryExecution.executedPlan // Check if SparkPlan Filter is not removed in a physical plan because JDBCRDD // cannot compile given predicates. - assert(parentPlan.isInstanceOf[org.apache.spark.sql.execution.WholeStageCodegen]) - val node = parentPlan.asInstanceOf[org.apache.spark.sql.execution.WholeStageCodegen] - assert(node.child.isInstanceOf[org.apache.spark.sql.execution.Filter]) + assert(parentPlan.isInstanceOf[org.apache.spark.sql.execution.WholeStageCodegenExec]) + val node = parentPlan.asInstanceOf[org.apache.spark.sql.execution.WholeStageCodegenExec] + assert(node.child.isInstanceOf[org.apache.spark.sql.execution.FilterExec]) df } assert(checkNotPushdown(sql("SELECT * FROM foobar WHERE (THEID + 1) < 2")).collect().size == 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 19e34b45bff6705fa757f847020ef025300a9a5c..14707774cf15cbf295a84abb3af15233094d19f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -312,7 +312,7 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext with Predic try { val queryExecution = sql(sqlString).queryExecution val rawPlan = queryExecution.executedPlan.collect { - case p: execution.DataSourceScan => p + case p: execution.DataSourceScanExec => p } match { case Seq(p) => p case _ => fail(s"More than one PhysicalRDD found\n$queryExecution") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index 62f991fc5dc6152aabfee6d87dc5f794c5f7bfc0..9bb901bfb3cfb1b06847b261b53a632973589a0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -124,7 +124,7 @@ class PrunedScanSuite extends DataSourceTest with SharedSQLContext { try { val queryExecution = sql(sqlString).queryExecution val rawPlan = queryExecution.executedPlan.collect { - case p: execution.DataSourceScan => p + case p: execution.DataSourceScanExec => p } match { case Seq(p) => p case _ => fail(s"More than one PhysicalRDD found\n$queryExecution") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index f6150198ddb4726bf86111a68c611c2724ac4372..5691105235f5b9ef0f1dfdb79686e91856ff8d88 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.Filter +import org.apache.spark.sql.execution.FilterExec import org.apache.spark.util.Utils /** @@ -242,7 +242,7 @@ private[sql] trait SQLTestUtils protected def stripSparkFilter(df: DataFrame): DataFrame = { val schema = df.schema val withoutFilters = df.queryExecution.sparkPlan transform { - case Filter(_, child) => child + case FilterExec(_, child) => child } val childRDD = withoutFilters diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index e7d2b5ad96821f3f0f9bfb7ad0583de7c8c48006..eb25ea06295e6f4e820b14542d9b28536e79b4dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.sql.{functions, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Project} -import org.apache.spark.sql.execution.{QueryExecution, WholeStageCodegen} +import org.apache.spark.sql.execution.{QueryExecution, WholeStageCodegenExec} import org.apache.spark.sql.test.SharedSQLContext class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { @@ -93,7 +93,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { val metric = qe.executedPlan match { - case w: WholeStageCodegen => w.child.longMetric("numOutputRows") + case w: WholeStageCodegenExec => w.child.longMetric("numOutputRows") case other => other.longMetric("numOutputRows") } metrics += metric.value.value diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index bc453340363322367ece6800e4c509c144bb17b8..f15f5b01e252e4b722a93c6a355fedd9b98fde2b 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -224,7 +224,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { val plan = statement.executeQuery("explain select * from test_table") plan.next() plan.next() - assert(plan.getString(1).contains("InMemoryColumnarTableScan")) + assert(plan.getString(1).contains("InMemoryTableScan")) val rs1 = statement.executeQuery("SELECT key FROM test_table ORDER BY KEY DESC") val buf1 = new collection.mutable.ArrayBuffer[Int]() @@ -310,7 +310,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { val plan = statement.executeQuery("explain select key from test_map ORDER BY key DESC") plan.next() plan.next() - assert(plan.getString(1).contains("InMemoryColumnarTableScan")) + assert(plan.getString(1).contains("InMemoryTableScan")) val rs = statement.executeQuery("SELECT key FROM test_map ORDER BY KEY DESC") val buf = new collection.mutable.ArrayBuffer[Int]() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 5b7fbe0ce54a02b9552878298f385a0ee6575a38..2d36ddafe60889c833ad7e238df9df1282b1e7ea 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -78,7 +78,7 @@ private[hive] trait HiveStrategies { projectList, otherPredicates, identity[Seq[Expression]], - HiveTableScan(_, relation, pruningPredicates)(context, hiveconf)) :: Nil + HiveTableScanExec(_, relation, pruningPredicates)(context, hiveconf)) :: Nil case _ => Nil } @@ -91,17 +91,17 @@ private[hive] trait HiveStrategies { val cmd = CreateMetastoreDataSource( tableIdent, userSpecifiedSchema, provider, opts, allowExisting, managedIfNoPath) - ExecutedCommand(cmd) :: Nil + ExecutedCommandExec(cmd) :: Nil case c: CreateTableUsingAsSelect if c.temporary => val cmd = CreateTempTableUsingAsSelect( c.tableIdent, c.provider, c.partitionColumns, c.mode, c.options, c.child) - ExecutedCommand(cmd) :: Nil + ExecutedCommandExec(cmd) :: Nil case c: CreateTableUsingAsSelect => val cmd = CreateMetastoreDataSourceAsSelect(c.tableIdent, c.provider, c.partitionColumns, c.bucketSpec, c.mode, c.options, c.child) - ExecutedCommand(cmd) :: Nil + ExecutedCommandExec(cmd) :: Nil case _ => Nil } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala similarity index 99% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala index 9a834660f953fe451b8a075f827f7ddef02dc660..0f720910967e759ee89dc5a3691485c18c269538 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -44,13 +44,13 @@ import org.apache.spark.util.Utils * @param partitionPruningPred An optional partition pruning predicate for partitioned table. */ private[hive] -case class HiveTableScan( +case class HiveTableScanExec( requestedAttributes: Seq[Attribute], relation: MetastoreRelation, partitionPruningPred: Seq[Expression])( @transient val context: SQLContext, @transient val hiveconf: HiveConf) - extends LeafNode { + extends LeafExecNode { require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, "Partition pruning predicates only supported for partitioned tables.") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index e614daadf391844fe067e4bb9b4553af22438672..3cb60816876d08f40e1fa32868d8cae96b22b404 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.mapred.{FileOutputFormat, JobConf} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.execution.{SparkPlan, UnaryNode} +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} import org.apache.spark.SparkException @@ -41,7 +41,7 @@ case class InsertIntoHiveTable( partition: Map[String, Option[String]], child: SparkPlan, overwrite: Boolean, - ifNotExists: Boolean) extends UnaryNode { + ifNotExists: Boolean) extends UnaryExecNode { @transient private val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] @transient private val client = sessionState.metadataHive diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 8c8becfb8752902ec421f5d1dbef136096ab9381..f27337eb36a64cefdb040fe1670896bcb73c154d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -59,7 +59,7 @@ case class ScriptTransformation( output: Seq[Attribute], child: SparkPlan, ioschema: HiveScriptIOSchema)(@transient private val hiveconf: HiveConf) - extends UnaryNode { + extends UnaryExecNode { override protected def otherCopyArgs: Seq[HiveConf] = hiveconf :: Nil diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 11384a0275ae3c4353a22d8fb39b570b442a0eb6..97bd47a247d2bd1dad0eab052b3ea7df564c65ac 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hive import java.io.File import org.apache.spark.sql.{AnalysisException, QueryTest, SaveMode} -import org.apache.spark.sql.execution.columnar.InMemoryColumnarTableScan +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.storage.RDDBlockId import org.apache.spark.util.Utils @@ -31,7 +31,7 @@ class CachedTableSuite extends QueryTest with TestHiveSingleton { def rddIdOf(tableName: String): Int = { val plan = table(tableName).queryExecution.sparkPlan plan.collect { - case InMemoryColumnarTableScan(_, _, relation) => + case InMemoryTableScanExec(_, _, relation) => relation.cachedColumnBuffers.id case _ => fail(s"Table $tableName is not cached\n" + plan) @@ -211,7 +211,7 @@ class CachedTableSuite extends QueryTest with TestHiveSingleton { cacheTable("cachedTable") val sparkPlan = sql("SELECT * FROM cachedTable").queryExecution.sparkPlan - assert(sparkPlan.collect { case e: InMemoryColumnarTableScan => e }.size === 1) + assert(sparkPlan.collect { case e: InMemoryTableScanExec => e }.size === 1) sql("DROP TABLE cachedTable") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 565b310bb7a0ed5a7f282909fe285d84c6253b6a..93a6f0bb58907659e6e106650f2adeab8fc850ab 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -153,7 +153,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { // Using `sparkPlan` because for relevant patterns in HashJoin to be // matched, other strategies need to be applied. - var bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } + var bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoinExec => j } assert(bhj.size === 1, s"actual query plans do not contain broadcast join: ${df.queryExecution}") @@ -164,10 +164,10 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1""") df = sql(query) - bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } + bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoinExec => j } assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") - val shj = df.queryExecution.sparkPlan.collect { case j: SortMergeJoin => j } + val shj = df.queryExecution.sparkPlan.collect { case j: SortMergeJoinExec => j } assert(shj.size === 1, "SortMergeJoin should be planned when BroadcastHashJoin is turned off") @@ -210,7 +210,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { // Using `sparkPlan` because for relevant patterns in HashJoin to be // matched, other strategies need to be applied. var bhj = df.queryExecution.sparkPlan.collect { - case j: BroadcastHashJoin => j + case j: BroadcastHashJoinExec => j } assert(bhj.size === 1, s"actual query plans do not contain broadcast join: ${df.queryExecution}") @@ -223,12 +223,12 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1") df = sql(leftSemiJoinQuery) bhj = df.queryExecution.sparkPlan.collect { - case j: BroadcastHashJoin => j + case j: BroadcastHashJoinExec => j } assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") val shj = df.queryExecution.sparkPlan.collect { - case j: ShuffledHashJoin => j + case j: ShuffledHashJoinExec => j } assert(shj.size === 1, "LeftSemiJoinHash should be planned when BroadcastHashJoin is turned off") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 3ddffeb14a2b56fd801132a454d66b81d6a847b9..aac5cc6d40c84e98ea1ac8569764567a8075e566 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -485,7 +485,7 @@ abstract class HiveComparisonTest // also print out the query plans and results for those. val computedTablesMessages: String = try { val tablesRead = new TestHiveQueryExecution(query).executedPlan.collect { - case ts: HiveTableScan => ts.relation.tableName + case ts: HiveTableScanExec => ts.relation.tableName }.toSet TestHive.reset() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 93d63f2241324e396701881df312c887eb3bb8b3..467a67259f4e1c621fa254655d16d4c17958284a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.{SparkException, SparkFiles} import org.apache.spark.sql.{AnalysisException, DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.catalyst.plans.logical.Project -import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoin +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext} import org.apache.spark.sql.hive.test.TestHive._ @@ -121,7 +121,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("SPARK-10484 Optimize the Cartesian (Cross) Join with broadcast based JOIN") { def assertBroadcastNestedLoopJoin(sqlText: String): Unit = { assert(sql(sqlText).queryExecution.sparkPlan.collect { - case _: BroadcastNestedLoopJoin => 1 + case _: BroadcastNestedLoopJoinExec => 1 }.nonEmpty) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala index 6b424d73430e24051fa3054ae318e114b007f77b..2de429bdabb77cccb9c4eb8d2bd87822a3bc69db 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.catalyst.expressions.{Cast, EqualTo} -import org.apache.spark.sql.execution.Project +import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.hive.test.TestHive /** @@ -50,7 +50,7 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { test("[SPARK-2210] boolean cast on boolean value should be removed") { val q = "select cast(cast(key=0 as boolean) as boolean) from src" val project = TestHive.sql(q).queryExecution.sparkPlan.collect { - case e: Project => e + case e: ProjectExec => e }.head // No cast expression introduced diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 12f30e2e746fac25a58bddab3f678e0e2667edb3..24df73b40ea0e899f7f58564bac5379552b19396 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -151,7 +151,7 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { val plan = new TestHiveQueryExecution(sql).sparkPlan val actualOutputColumns = plan.output.map(_.name) val (actualScannedColumns, actualPartValues) = plan.collect { - case p @ HiveTableScan(columns, relation, _) => + case p @ HiveTableScanExec(columns, relation, _) => val columnNames = columns.map(_.name) val partValues = if (relation.catalogTable.partitionColumnNames.nonEmpty) { p.prunePartitions(relation.getHiveQlPartitions()).map(_.getValues) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index 00b5c8dd41730a1237ffc4094026129c032ed270..1a15fb741a4babe93974d70bece6dd5a6cc1da85 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest, UnaryNode} +import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest, UnaryExecNode} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.types.StringType @@ -111,7 +111,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { } } -private case class ExceptionInjectingOperator(child: SparkPlan) extends UnaryNode { +private case class ExceptionInjectingOperator(child: SparkPlan) extends UnaryExecNode { override protected def doExecute(): RDD[InternalRow] = { child.execute().map { x => assert(TaskContext.get() != null) // Make sure that TaskContext is defined. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 2984ee99bee779f7471e0e8a9ab52b64d8615648..1c1f6d910d63d64c6824363cd7fcedcfbf4b463d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -21,10 +21,10 @@ import java.io.File import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.execution.DataSourceScan -import org.apache.spark.sql.execution.command.ExecutedCommand +import org.apache.spark.sql.execution.DataSourceScanExec +import org.apache.spark.sql.execution.command.ExecutedCommandExec import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.hive.execution.HiveTableScan +import org.apache.spark.sql.hive.execution.HiveTableScanExec import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -192,11 +192,11 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { test(s"conversion is working") { assert( sql("SELECT * FROM normal_parquet").queryExecution.sparkPlan.collect { - case _: HiveTableScan => true + case _: HiveTableScanExec => true }.isEmpty) assert( sql("SELECT * FROM normal_parquet").queryExecution.sparkPlan.collect { - case _: DataSourceScan => true + case _: DataSourceScanExec => true }.nonEmpty) } @@ -307,7 +307,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt") df.queryExecution.sparkPlan match { - case ExecutedCommand(_: InsertIntoHadoopFsRelation) => // OK + case ExecutedCommandExec(_: InsertIntoHadoopFsRelation) => // OK case o => fail("test_insert_parquet should be converted to a " + s"${classOf[HadoopFsRelation ].getCanonicalName} and " + s"${classOf[InsertIntoDataSource].getCanonicalName} is expected as the SparkPlan. " + @@ -337,7 +337,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt_array") df.queryExecution.sparkPlan match { - case ExecutedCommand(_: InsertIntoHadoopFsRelation) => // OK + case ExecutedCommandExec(_: InsertIntoHadoopFsRelation) => // OK case o => fail("test_insert_parquet should be converted to a " + s"${classOf[HadoopFsRelation ].getCanonicalName} and " + s"${classOf[InsertIntoDataSource].getCanonicalName} is expected as the SparkPlan." + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index aa6101f7b73cfb4d510b019b8faa51bc4ca6ad29..d271e55467c6fb3bd189fb8831c5ce933b84b314 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -22,10 +22,10 @@ import java.io.File import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.DataSourceScan +import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.datasources.{BucketSpec, DataSourceStrategy} import org.apache.spark.sql.execution.exchange.ShuffleExchange -import org.apache.spark.sql.execution.joins.SortMergeJoin +import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -93,7 +93,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet // Filter could hide the bug in bucket pruning. Thus, skipping all the filters val plan = bucketedDataFrame.filter(filterCondition).queryExecution.executedPlan - val rdd = plan.find(_.isInstanceOf[DataSourceScan]) + val rdd = plan.find(_.isInstanceOf[DataSourceScanExec]) assert(rdd.isDefined, plan) val checkedResult = rdd.get.execute().mapPartitionsWithIndex { case (index, iter) => @@ -261,8 +261,8 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet joined.sort("bucketed_table1.k", "bucketed_table2.k"), df1.join(df2, joinCondition(df1, df2, joinColumns)).sort("df1.k", "df2.k")) - assert(joined.queryExecution.executedPlan.isInstanceOf[SortMergeJoin]) - val joinOperator = joined.queryExecution.executedPlan.asInstanceOf[SortMergeJoin] + assert(joined.queryExecution.executedPlan.isInstanceOf[SortMergeJoinExec]) + val joinOperator = joined.queryExecution.executedPlan.asInstanceOf[SortMergeJoinExec] assert( joinOperator.left.find(_.isInstanceOf[ShuffleExchange]).isDefined == shuffleLeft, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala index a15bd227a920100d37c0203679ee5a736edf5921..19749a97132b3be24fd75f5545bb0eff009a330a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala @@ -152,8 +152,8 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { val df = sqlContext.read.parquet(path).filter('a === 0).select('b) val physicalPlan = df.queryExecution.sparkPlan - assert(physicalPlan.collect { case p: execution.Project => p }.length === 1) - assert(physicalPlan.collect { case p: execution.Filter => p }.length === 1) + assert(physicalPlan.collect { case p: execution.ProjectExec => p }.length === 1) + assert(physicalPlan.collect { case p: execution.FilterExec => p }.length === 1) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala index 089cef615fa2adf22c43941f853ff52996d74e6b..5378336ff8a907fef11e07f8828cf4a3b8300736 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala @@ -28,7 +28,7 @@ import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ -import org.apache.spark.sql.execution.DataSourceScan +import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.datasources.{FileScanRDD, HadoopFsRelation, LocalityTestFileSystem, LogicalRelation} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -688,7 +688,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes .load(path) val Some(fileScanRDD) = df2.queryExecution.executedPlan.collectFirst { - case scan: DataSourceScan if scan.rdd.isInstanceOf[FileScanRDD] => + case scan: DataSourceScanExec if scan.rdd.isInstanceOf[FileScanRDD] => scan.rdd.asInstanceOf[FileScanRDD] }