You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2016/04/23 02:44:01 UTC

[1/7] spark git commit: [SPARK-14855][SQL] Add "Exec" suffix to physical operators

Repository: spark
Updated Branches:
  refs/heads/master c431a76d0 -> d7d0cad0a


http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala
----------------------------------------------------------------------
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 62f991f..9bb901b 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")

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
----------------------------------------------------------------------
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 f615019..5691105 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala
----------------------------------------------------------------------
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 e7d2b5a..eb25ea0 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
----------------------------------------------------------------------
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 bc45334..f15f5b0 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]()

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
----------------------------------------------------------------------
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 5b7fbe0..2d36dda 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
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
----------------------------------------------------------------------
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/HiveTableScan.scala
deleted file mode 100644
index 9a83466..0000000
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
+++ /dev/null
@@ -1,166 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive.execution
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.hive.conf.HiveConf
-import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition}
-import org.apache.hadoop.hive.serde.serdeConstants
-import org.apache.hadoop.hive.serde2.objectinspector._
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SQLContext
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.execution._
-import org.apache.spark.sql.execution.metric.SQLMetrics
-import org.apache.spark.sql.hive._
-import org.apache.spark.sql.types.{BooleanType, DataType}
-import org.apache.spark.util.Utils
-
-/**
- * The Hive table scan operator.  Column and partition pruning are both handled.
- *
- * @param requestedAttributes Attributes to be fetched from the Hive table.
- * @param relation The Hive table be be scanned.
- * @param partitionPruningPred An optional partition pruning predicate for partitioned table.
- */
-private[hive]
-case class HiveTableScan(
-    requestedAttributes: Seq[Attribute],
-    relation: MetastoreRelation,
-    partitionPruningPred: Seq[Expression])(
-    @transient val context: SQLContext,
-    @transient val hiveconf: HiveConf)
-  extends LeafNode {
-
-  require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned,
-    "Partition pruning predicates only supported for partitioned tables.")
-
-  private[sql] override lazy val metrics = Map(
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
-
-  override def producedAttributes: AttributeSet = outputSet ++
-    AttributeSet(partitionPruningPred.flatMap(_.references))
-
-  // Retrieve the original attributes based on expression ID so that capitalization matches.
-  val attributes = requestedAttributes.map(relation.attributeMap)
-
-  // Bind all partition key attribute references in the partition pruning predicate for later
-  // evaluation.
-  private[this] val boundPruningPred = partitionPruningPred.reduceLeftOption(And).map { pred =>
-    require(
-      pred.dataType == BooleanType,
-      s"Data type of predicate $pred must be BooleanType rather than ${pred.dataType}.")
-
-    BindReferences.bindReference(pred, relation.partitionKeys)
-  }
-
-  // Create a local copy of hiveconf,so that scan specific modifications should not impact
-  // other queries
-  @transient
-  private[this] val hiveExtraConf = new HiveConf(hiveconf)
-
-  // append columns ids and names before broadcast
-  addColumnMetadataToConf(hiveExtraConf)
-
-  @transient
-  private[this] val hadoopReader =
-    new HadoopTableReader(attributes, relation, context, hiveExtraConf)
-
-  private[this] def castFromString(value: String, dataType: DataType) = {
-    Cast(Literal(value), dataType).eval(null)
-  }
-
-  private def addColumnMetadataToConf(hiveConf: HiveConf) {
-    // Specifies needed column IDs for those non-partitioning columns.
-    val neededColumnIDs = attributes.flatMap(relation.columnOrdinals.get).map(o => o: Integer)
-
-    HiveShim.appendReadColumns(hiveConf, neededColumnIDs, attributes.map(_.name))
-
-    val tableDesc = relation.tableDesc
-    val deserializer = tableDesc.getDeserializerClass.newInstance
-    deserializer.initialize(hiveConf, tableDesc.getProperties)
-
-    // Specifies types and object inspectors of columns to be scanned.
-    val structOI = ObjectInspectorUtils
-      .getStandardObjectInspector(
-        deserializer.getObjectInspector,
-        ObjectInspectorCopyOption.JAVA)
-      .asInstanceOf[StructObjectInspector]
-
-    val columnTypeNames = structOI
-      .getAllStructFieldRefs.asScala
-      .map(_.getFieldObjectInspector)
-      .map(TypeInfoUtils.getTypeInfoFromObjectInspector(_).getTypeName)
-      .mkString(",")
-
-    hiveConf.set(serdeConstants.LIST_COLUMN_TYPES, columnTypeNames)
-    hiveConf.set(serdeConstants.LIST_COLUMNS, relation.attributes.map(_.name).mkString(","))
-  }
-
-  /**
-   * Prunes partitions not involve the query plan.
-   *
-   * @param partitions All partitions of the relation.
-   * @return Partitions that are involved in the query plan.
-   */
-  private[hive] def prunePartitions(partitions: Seq[HivePartition]) = {
-    boundPruningPred match {
-      case None => partitions
-      case Some(shouldKeep) => partitions.filter { part =>
-        val dataTypes = relation.partitionKeys.map(_.dataType)
-        val castedValues = part.getValues.asScala.zip(dataTypes)
-          .map { case (value, dataType) => castFromString(value, dataType) }
-
-        // Only partitioned values are needed here, since the predicate has already been bound to
-        // partition key attribute references.
-        val row = InternalRow.fromSeq(castedValues)
-        shouldKeep.eval(row).asInstanceOf[Boolean]
-      }
-    }
-  }
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    // Using dummyCallSite, as getCallSite can turn out to be expensive with
-    // with multiple partitions.
-    val rdd = if (!relation.hiveQlTable.isPartitioned) {
-      Utils.withDummyCallSite(sqlContext.sparkContext) {
-        hadoopReader.makeRDDForTable(relation.hiveQlTable)
-      }
-    } else {
-      Utils.withDummyCallSite(sqlContext.sparkContext) {
-        hadoopReader.makeRDDForPartitionedTable(
-          prunePartitions(relation.getHiveQlPartitions(partitionPruningPred)))
-      }
-    }
-    val numOutputRows = longMetric("numOutputRows")
-    rdd.mapPartitionsInternal { iter =>
-      val proj = UnsafeProjection.create(schema)
-      iter.map { r =>
-        numOutputRows += 1
-        proj(r)
-      }
-    }
-  }
-
-  override def output: Seq[Attribute] = attributes
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala
new file mode 100644
index 0000000..0f72091
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive.execution
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.hive.conf.HiveConf
+import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition}
+import org.apache.hadoop.hive.serde.serdeConstants
+import org.apache.hadoop.hive.serde2.objectinspector._
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.metric.SQLMetrics
+import org.apache.spark.sql.hive._
+import org.apache.spark.sql.types.{BooleanType, DataType}
+import org.apache.spark.util.Utils
+
+/**
+ * The Hive table scan operator.  Column and partition pruning are both handled.
+ *
+ * @param requestedAttributes Attributes to be fetched from the Hive table.
+ * @param relation The Hive table be be scanned.
+ * @param partitionPruningPred An optional partition pruning predicate for partitioned table.
+ */
+private[hive]
+case class HiveTableScanExec(
+    requestedAttributes: Seq[Attribute],
+    relation: MetastoreRelation,
+    partitionPruningPred: Seq[Expression])(
+    @transient val context: SQLContext,
+    @transient val hiveconf: HiveConf)
+  extends LeafExecNode {
+
+  require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned,
+    "Partition pruning predicates only supported for partitioned tables.")
+
+  private[sql] override lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
+
+  override def producedAttributes: AttributeSet = outputSet ++
+    AttributeSet(partitionPruningPred.flatMap(_.references))
+
+  // Retrieve the original attributes based on expression ID so that capitalization matches.
+  val attributes = requestedAttributes.map(relation.attributeMap)
+
+  // Bind all partition key attribute references in the partition pruning predicate for later
+  // evaluation.
+  private[this] val boundPruningPred = partitionPruningPred.reduceLeftOption(And).map { pred =>
+    require(
+      pred.dataType == BooleanType,
+      s"Data type of predicate $pred must be BooleanType rather than ${pred.dataType}.")
+
+    BindReferences.bindReference(pred, relation.partitionKeys)
+  }
+
+  // Create a local copy of hiveconf,so that scan specific modifications should not impact
+  // other queries
+  @transient
+  private[this] val hiveExtraConf = new HiveConf(hiveconf)
+
+  // append columns ids and names before broadcast
+  addColumnMetadataToConf(hiveExtraConf)
+
+  @transient
+  private[this] val hadoopReader =
+    new HadoopTableReader(attributes, relation, context, hiveExtraConf)
+
+  private[this] def castFromString(value: String, dataType: DataType) = {
+    Cast(Literal(value), dataType).eval(null)
+  }
+
+  private def addColumnMetadataToConf(hiveConf: HiveConf) {
+    // Specifies needed column IDs for those non-partitioning columns.
+    val neededColumnIDs = attributes.flatMap(relation.columnOrdinals.get).map(o => o: Integer)
+
+    HiveShim.appendReadColumns(hiveConf, neededColumnIDs, attributes.map(_.name))
+
+    val tableDesc = relation.tableDesc
+    val deserializer = tableDesc.getDeserializerClass.newInstance
+    deserializer.initialize(hiveConf, tableDesc.getProperties)
+
+    // Specifies types and object inspectors of columns to be scanned.
+    val structOI = ObjectInspectorUtils
+      .getStandardObjectInspector(
+        deserializer.getObjectInspector,
+        ObjectInspectorCopyOption.JAVA)
+      .asInstanceOf[StructObjectInspector]
+
+    val columnTypeNames = structOI
+      .getAllStructFieldRefs.asScala
+      .map(_.getFieldObjectInspector)
+      .map(TypeInfoUtils.getTypeInfoFromObjectInspector(_).getTypeName)
+      .mkString(",")
+
+    hiveConf.set(serdeConstants.LIST_COLUMN_TYPES, columnTypeNames)
+    hiveConf.set(serdeConstants.LIST_COLUMNS, relation.attributes.map(_.name).mkString(","))
+  }
+
+  /**
+   * Prunes partitions not involve the query plan.
+   *
+   * @param partitions All partitions of the relation.
+   * @return Partitions that are involved in the query plan.
+   */
+  private[hive] def prunePartitions(partitions: Seq[HivePartition]) = {
+    boundPruningPred match {
+      case None => partitions
+      case Some(shouldKeep) => partitions.filter { part =>
+        val dataTypes = relation.partitionKeys.map(_.dataType)
+        val castedValues = part.getValues.asScala.zip(dataTypes)
+          .map { case (value, dataType) => castFromString(value, dataType) }
+
+        // Only partitioned values are needed here, since the predicate has already been bound to
+        // partition key attribute references.
+        val row = InternalRow.fromSeq(castedValues)
+        shouldKeep.eval(row).asInstanceOf[Boolean]
+      }
+    }
+  }
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    // Using dummyCallSite, as getCallSite can turn out to be expensive with
+    // with multiple partitions.
+    val rdd = if (!relation.hiveQlTable.isPartitioned) {
+      Utils.withDummyCallSite(sqlContext.sparkContext) {
+        hadoopReader.makeRDDForTable(relation.hiveQlTable)
+      }
+    } else {
+      Utils.withDummyCallSite(sqlContext.sparkContext) {
+        hadoopReader.makeRDDForPartitionedTable(
+          prunePartitions(relation.getHiveQlPartitions(partitionPruningPred)))
+      }
+    }
+    val numOutputRows = longMetric("numOutputRows")
+    rdd.mapPartitionsInternal { iter =>
+      val proj = UnsafeProjection.create(schema)
+      iter.map { r =>
+        numOutputRows += 1
+        proj(r)
+      }
+    }
+  }
+
+  override def output: Seq[Attribute] = attributes
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
----------------------------------------------------------------------
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 e614daa..3cb6081 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
----------------------------------------------------------------------
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 8c8becf..f27337e 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
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
----------------------------------------------------------------------
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 11384a0..97bd47a 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")
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
----------------------------------------------------------------------
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 565b310..93a6f0b 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")

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
----------------------------------------------------------------------
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 3ddffeb..aac5cc6 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()

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
----------------------------------------------------------------------
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 93d63f2..467a672 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)
     }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
----------------------------------------------------------------------
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 6b424d7..2de429b 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
----------------------------------------------------------------------
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 12f30e2..24df73b 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)

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala
----------------------------------------------------------------------
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 00b5c8d..1a15fb7 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.

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
----------------------------------------------------------------------
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 2984ee9..1c1f6d9 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." +

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala
----------------------------------------------------------------------
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 aa6101f..d271e55 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,

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala
----------------------------------------------------------------------
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 a15bd22..19749a9 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)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala
----------------------------------------------------------------------
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 089cef6..5378336 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]
           }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[4/7] spark git commit: [SPARK-14855][SQL] Add "Exec" suffix to physical operators

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala
----------------------------------------------------------------------
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 45a3213..971770a 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
----------------------------------------------------------------------
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 ac3c52e..9bebd74 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))
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
----------------------------------------------------------------------
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 c1a97de..751daa0 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
----------------------------------------------------------------------
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 e6079ec..5b96ab1 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]] {

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala
----------------------------------------------------------------------
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/BroadcastExchange.scala
deleted file mode 100644
index 87a113e..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.exchange
-
-import scala.concurrent.{ExecutionContext, Future}
-import scala.concurrent.duration._
-
-import org.apache.spark.broadcast
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.UnsafeRow
-import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, BroadcastPartitioning, Partitioning}
-import org.apache.spark.sql.execution.{SparkPlan, SQLExecution}
-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.
- */
-case class BroadcastExchange(
-    mode: BroadcastMode,
-    child: SparkPlan) extends Exchange {
-
-  override private[sql] lazy val metrics = Map(
-    "dataSize" -> SQLMetrics.createLongMetric(sparkContext, "data size (bytes)"),
-    "collectTime" -> SQLMetrics.createLongMetric(sparkContext, "time to collect (ms)"),
-    "buildTime" -> SQLMetrics.createLongMetric(sparkContext, "time to build (ms)"),
-    "broadcastTime" -> SQLMetrics.createLongMetric(sparkContext, "time to broadcast (ms)"))
-
-  override def outputPartitioning: Partitioning = BroadcastPartitioning(mode)
-
-  override def sameResult(plan: SparkPlan): Boolean = plan match {
-    case p: BroadcastExchange =>
-      mode.compatibleWith(p.mode) && child.sameResult(p.child)
-    case _ => false
-  }
-
-  @transient
-  private val timeout: Duration = {
-    val timeoutValue = sqlContext.conf.broadcastTimeout
-    if (timeoutValue < 0) {
-      Duration.Inf
-    } else {
-      timeoutValue.seconds
-    }
-  }
-
-  @transient
-  private lazy val relationFuture: Future[broadcast.Broadcast[Any]] = {
-    // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here.
-    val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
-    Future {
-      // This will run in another thread. Set the execution id so that we can connect these jobs
-      // with the correct execution.
-      SQLExecution.withExecutionId(sparkContext, executionId) {
-        val beforeCollect = System.nanoTime()
-        // Note that we use .executeCollect() because we don't want to convert data to Scala types
-        val input: Array[InternalRow] = child.executeCollect()
-        val beforeBuild = System.nanoTime()
-        longMetric("collectTime") += (beforeBuild - beforeCollect) / 1000000
-        longMetric("dataSize") += input.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum
-
-        // Construct and broadcast the relation.
-        val relation = mode.transform(input)
-        val beforeBroadcast = System.nanoTime()
-        longMetric("buildTime") += (beforeBroadcast - beforeBuild) / 1000000
-
-        val broadcasted = sparkContext.broadcast(relation)
-        longMetric("broadcastTime") += (System.nanoTime() - beforeBroadcast) / 1000000
-        broadcasted
-      }
-    }(BroadcastExchange.executionContext)
-  }
-
-  override protected def doPrepare(): Unit = {
-    // Materialize the future.
-    relationFuture
-  }
-
-  override protected def doExecute(): RDD[InternalRow] = {
-    throw new UnsupportedOperationException(
-      "BroadcastExchange does not support the execute() code path.")
-  }
-
-  override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = {
-    ThreadUtils.awaitResult(relationFuture, timeout).asInstanceOf[broadcast.Broadcast[T]]
-  }
-}
-
-object BroadcastExchange {
-  private[execution] val executionContext = ExecutionContext.fromExecutorService(
-    ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", 128))
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala
new file mode 100644
index 0000000..573ca19
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.exchange
+
+import scala.concurrent.{ExecutionContext, Future}
+import scala.concurrent.duration._
+
+import org.apache.spark.broadcast
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, BroadcastPartitioning, Partitioning}
+import org.apache.spark.sql.execution.{SparkPlan, SQLExecution}
+import org.apache.spark.sql.execution.metric.SQLMetrics
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * A [[BroadcastExchangeExec]] collects, transforms and finally broadcasts the result of
+ * a transformed SparkPlan.
+ */
+case class BroadcastExchangeExec(
+    mode: BroadcastMode,
+    child: SparkPlan) extends Exchange {
+
+  override private[sql] lazy val metrics = Map(
+    "dataSize" -> SQLMetrics.createLongMetric(sparkContext, "data size (bytes)"),
+    "collectTime" -> SQLMetrics.createLongMetric(sparkContext, "time to collect (ms)"),
+    "buildTime" -> SQLMetrics.createLongMetric(sparkContext, "time to build (ms)"),
+    "broadcastTime" -> SQLMetrics.createLongMetric(sparkContext, "time to broadcast (ms)"))
+
+  override def outputPartitioning: Partitioning = BroadcastPartitioning(mode)
+
+  override def sameResult(plan: SparkPlan): Boolean = plan match {
+    case p: BroadcastExchangeExec =>
+      mode.compatibleWith(p.mode) && child.sameResult(p.child)
+    case _ => false
+  }
+
+  @transient
+  private val timeout: Duration = {
+    val timeoutValue = sqlContext.conf.broadcastTimeout
+    if (timeoutValue < 0) {
+      Duration.Inf
+    } else {
+      timeoutValue.seconds
+    }
+  }
+
+  @transient
+  private lazy val relationFuture: Future[broadcast.Broadcast[Any]] = {
+    // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here.
+    val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
+    Future {
+      // This will run in another thread. Set the execution id so that we can connect these jobs
+      // with the correct execution.
+      SQLExecution.withExecutionId(sparkContext, executionId) {
+        val beforeCollect = System.nanoTime()
+        // Note that we use .executeCollect() because we don't want to convert data to Scala types
+        val input: Array[InternalRow] = child.executeCollect()
+        val beforeBuild = System.nanoTime()
+        longMetric("collectTime") += (beforeBuild - beforeCollect) / 1000000
+        longMetric("dataSize") += input.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum
+
+        // Construct and broadcast the relation.
+        val relation = mode.transform(input)
+        val beforeBroadcast = System.nanoTime()
+        longMetric("buildTime") += (beforeBroadcast - beforeBuild) / 1000000
+
+        val broadcasted = sparkContext.broadcast(relation)
+        longMetric("broadcastTime") += (System.nanoTime() - beforeBroadcast) / 1000000
+        broadcasted
+      }
+    }(BroadcastExchangeExec.executionContext)
+  }
+
+  override protected def doPrepare(): Unit = {
+    // Materialize the future.
+    relationFuture
+  }
+
+  override protected def doExecute(): RDD[InternalRow] = {
+    throw new UnsupportedOperationException(
+      "BroadcastExchange does not support the execute() code path.")
+  }
+
+  override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = {
+    ThreadUtils.awaitResult(relationFuture, timeout).asInstanceOf[broadcast.Broadcast[T]]
+  }
+}
+
+object BroadcastExchangeExec {
+  private[execution] val executionContext = ExecutionContext.fromExecutorService(
+    ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", 128))
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
----------------------------------------------------------------------
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 4864db7..446571a 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
         }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala
----------------------------------------------------------------------
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 df7ad48..9da9df6 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala
----------------------------------------------------------------------
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/BroadcastHashJoin.scala
deleted file mode 100644
index 89487c6..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala
+++ /dev/null
@@ -1,401 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.joins
-
-import org.apache.spark.TaskContext
-import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.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.metric.SQLMetrics
-import org.apache.spark.sql.types.LongType
-
-/**
- * Performs an inner hash join of two child relations.  When the output RDD of this operator is
- * being constructed, a Spark job is asynchronously started to calculate the values for the
- * broadcast relation.  This data is then placed in a Spark broadcast variable.  The streamed
- * relation is not shuffled.
- */
-case class BroadcastHashJoin(
-    leftKeys: Seq[Expression],
-    rightKeys: Seq[Expression],
-    joinType: JoinType,
-    buildSide: BuildSide,
-    condition: Option[Expression],
-    left: SparkPlan,
-    right: SparkPlan)
-  extends BinaryNode with HashJoin with CodegenSupport {
-
-  override private[sql] lazy val metrics = Map(
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
-
-  override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning
-
-  override def requiredChildDistribution: Seq[Distribution] = {
-    val mode = HashedRelationBroadcastMode(buildKeys)
-    buildSide match {
-      case BuildLeft =>
-        BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil
-      case BuildRight =>
-        UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil
-    }
-  }
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    val numOutputRows = longMetric("numOutputRows")
-
-    val broadcastRelation = buildPlan.executeBroadcast[HashedRelation]()
-    streamedPlan.execute().mapPartitions { streamedIter =>
-      val hashed = broadcastRelation.value.asReadOnlyCopy()
-      TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize)
-      join(streamedIter, hashed, numOutputRows)
-    }
-  }
-
-  override def inputRDDs(): Seq[RDD[InternalRow]] = {
-    streamedPlan.asInstanceOf[CodegenSupport].inputRDDs()
-  }
-
-  override def doProduce(ctx: CodegenContext): String = {
-    streamedPlan.asInstanceOf[CodegenSupport].produce(ctx, this)
-  }
-
-  override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
-    joinType match {
-      case Inner => codegenInner(ctx, input)
-      case LeftOuter | RightOuter => codegenOuter(ctx, input)
-      case LeftSemi => codegenSemi(ctx, input)
-      case LeftAnti => codegenAnti(ctx, input)
-      case x =>
-        throw new IllegalArgumentException(
-          s"BroadcastHashJoin should not take $x as the JoinType")
-    }
-  }
-
-  /**
-   * Returns a tuple of Broadcast of HashedRelation and the variable name for it.
-   */
-  private def prepareBroadcast(ctx: CodegenContext): (Broadcast[HashedRelation], String) = {
-    // create a name for HashedRelation
-    val broadcastRelation = buildPlan.executeBroadcast[HashedRelation]()
-    val broadcast = ctx.addReferenceObj("broadcast", broadcastRelation)
-    val relationTerm = ctx.freshName("relation")
-    val clsName = broadcastRelation.value.getClass.getName
-    ctx.addMutableState(clsName, relationTerm,
-      s"""
-         | $relationTerm = (($clsName) $broadcast.value()).asReadOnlyCopy();
-         | incPeakExecutionMemory($relationTerm.estimatedSize());
-       """.stripMargin)
-    (broadcastRelation, relationTerm)
-  }
-
-  /**
-   * Returns the code for generating join key for stream side, and expression of whether the key
-   * has any null in it or not.
-   */
-  private def genStreamSideJoinKey(
-      ctx: CodegenContext,
-      input: Seq[ExprCode]): (ExprCode, String) = {
-    ctx.currentVars = input
-    if (streamedKeys.length == 1 && streamedKeys.head.dataType == LongType) {
-      // generate the join key as Long
-      val ev = streamedKeys.head.genCode(ctx)
-      (ev, ev.isNull)
-    } else {
-      // generate the join key as UnsafeRow
-      val ev = GenerateUnsafeProjection.createCode(ctx, streamedKeys)
-      (ev, s"${ev.value}.anyNull()")
-    }
-  }
-
-  /**
-   * Generates the code for variable of build side.
-   */
-  private def genBuildSideVars(ctx: CodegenContext, matched: String): Seq[ExprCode] = {
-    ctx.currentVars = null
-    ctx.INPUT_ROW = matched
-    buildPlan.output.zipWithIndex.map { case (a, i) =>
-      val ev = BoundReference(i, a.dataType, a.nullable).genCode(ctx)
-      if (joinType == Inner) {
-        ev
-      } else {
-        // the variables are needed even there is no matched rows
-        val isNull = ctx.freshName("isNull")
-        val value = ctx.freshName("value")
-        val code = s"""
-          |boolean $isNull = true;
-          |${ctx.javaType(a.dataType)} $value = ${ctx.defaultValue(a.dataType)};
-          |if ($matched != null) {
-          |  ${ev.code}
-          |  $isNull = ${ev.isNull};
-          |  $value = ${ev.value};
-          |}
-         """.stripMargin
-        ExprCode(code, isNull, value)
-      }
-    }
-  }
-
-  /**
-   * Generate the (non-equi) condition used to filter joined rows. This is used in Inner, Left Semi
-   * and Left Anti joins.
-   */
-  private def getJoinCondition(
-      ctx: CodegenContext,
-      input: Seq[ExprCode]): (String, String, Seq[ExprCode]) = {
-    val matched = ctx.freshName("matched")
-    val buildVars = genBuildSideVars(ctx, matched)
-    val checkCondition = if (condition.isDefined) {
-      val expr = condition.get
-      // evaluate the variables from build side that used by condition
-      val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references)
-      // filter the output via condition
-      ctx.currentVars = input ++ buildVars
-      val ev =
-        BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx)
-      s"""
-         |$eval
-         |${ev.code}
-         |if (${ev.isNull} || !${ev.value}) continue;
-       """.stripMargin
-    } else {
-      ""
-    }
-    (matched, checkCondition, buildVars)
-  }
-
-  /**
-   * Generates the code for Inner join.
-   */
-  private def codegenInner(ctx: CodegenContext, input: Seq[ExprCode]): String = {
-    val (broadcastRelation, relationTerm) = prepareBroadcast(ctx)
-    val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input)
-    val (matched, checkCondition, buildVars) = getJoinCondition(ctx, input)
-    val numOutput = metricTerm(ctx, "numOutputRows")
-
-    val resultVars = buildSide match {
-      case BuildLeft => buildVars ++ input
-      case BuildRight => input ++ buildVars
-    }
-    if (broadcastRelation.value.keyIsUnique) {
-      s"""
-         |// generate join key for stream side
-         |${keyEv.code}
-         |// find matches from HashedRelation
-         |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value});
-         |if ($matched == null) continue;
-         |$checkCondition
-         |$numOutput.add(1);
-         |${consume(ctx, resultVars)}
-       """.stripMargin
-
-    } else {
-      ctx.copyResult = true
-      val matches = ctx.freshName("matches")
-      val iteratorCls = classOf[Iterator[UnsafeRow]].getName
-      s"""
-         |// generate join key for stream side
-         |${keyEv.code}
-         |// find matches from HashRelation
-         |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value});
-         |if ($matches == null) continue;
-         |while ($matches.hasNext()) {
-         |  UnsafeRow $matched = (UnsafeRow) $matches.next();
-         |  $checkCondition
-         |  $numOutput.add(1);
-         |  ${consume(ctx, resultVars)}
-         |}
-       """.stripMargin
-    }
-  }
-
-  /**
-   * Generates the code for left or right outer join.
-   */
-  private def codegenOuter(ctx: CodegenContext, input: Seq[ExprCode]): String = {
-    val (broadcastRelation, relationTerm) = prepareBroadcast(ctx)
-    val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input)
-    val matched = ctx.freshName("matched")
-    val buildVars = genBuildSideVars(ctx, matched)
-    val numOutput = metricTerm(ctx, "numOutputRows")
-
-    // filter the output via condition
-    val conditionPassed = ctx.freshName("conditionPassed")
-    val checkCondition = if (condition.isDefined) {
-      val expr = condition.get
-      // evaluate the variables from build side that used by condition
-      val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references)
-      ctx.currentVars = input ++ buildVars
-      val ev =
-        BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx)
-      s"""
-         |boolean $conditionPassed = true;
-         |${eval.trim}
-         |${ev.code}
-         |if ($matched != null) {
-         |  $conditionPassed = !${ev.isNull} && ${ev.value};
-         |}
-       """.stripMargin
-    } else {
-      s"final boolean $conditionPassed = true;"
-    }
-
-    val resultVars = buildSide match {
-      case BuildLeft => buildVars ++ input
-      case BuildRight => input ++ buildVars
-    }
-    if (broadcastRelation.value.keyIsUnique) {
-      s"""
-         |// generate join key for stream side
-         |${keyEv.code}
-         |// find matches from HashedRelation
-         |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value});
-         |${checkCondition.trim}
-         |if (!$conditionPassed) {
-         |  $matched = null;
-         |  // reset the variables those are already evaluated.
-         |  ${buildVars.filter(_.code == "").map(v => s"${v.isNull} = true;").mkString("\n")}
-         |}
-         |$numOutput.add(1);
-         |${consume(ctx, resultVars)}
-       """.stripMargin
-
-    } else {
-      ctx.copyResult = true
-      val matches = ctx.freshName("matches")
-      val iteratorCls = classOf[Iterator[UnsafeRow]].getName
-      val found = ctx.freshName("found")
-      s"""
-         |// generate join key for stream side
-         |${keyEv.code}
-         |// find matches from HashRelation
-         |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value});
-         |boolean $found = false;
-         |// the last iteration of this loop is to emit an empty row if there is no matched rows.
-         |while ($matches != null && $matches.hasNext() || !$found) {
-         |  UnsafeRow $matched = $matches != null && $matches.hasNext() ?
-         |    (UnsafeRow) $matches.next() : null;
-         |  ${checkCondition.trim}
-         |  if (!$conditionPassed) continue;
-         |  $found = true;
-         |  $numOutput.add(1);
-         |  ${consume(ctx, resultVars)}
-         |}
-       """.stripMargin
-    }
-  }
-
-  /**
-   * Generates the code for left semi join.
-   */
-  private def codegenSemi(ctx: CodegenContext, input: Seq[ExprCode]): String = {
-    val (broadcastRelation, relationTerm) = prepareBroadcast(ctx)
-    val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input)
-    val (matched, checkCondition, _) = getJoinCondition(ctx, input)
-    val numOutput = metricTerm(ctx, "numOutputRows")
-    if (broadcastRelation.value.keyIsUnique) {
-      s"""
-         |// generate join key for stream side
-         |${keyEv.code}
-         |// find matches from HashedRelation
-         |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value});
-         |if ($matched == null) continue;
-         |$checkCondition
-         |$numOutput.add(1);
-         |${consume(ctx, input)}
-       """.stripMargin
-    } else {
-      val matches = ctx.freshName("matches")
-      val iteratorCls = classOf[Iterator[UnsafeRow]].getName
-      val found = ctx.freshName("found")
-      s"""
-         |// generate join key for stream side
-         |${keyEv.code}
-         |// find matches from HashRelation
-         |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value});
-         |if ($matches == null) continue;
-         |boolean $found = false;
-         |while (!$found && $matches.hasNext()) {
-         |  UnsafeRow $matched = (UnsafeRow) $matches.next();
-         |  $checkCondition
-         |  $found = true;
-         |}
-         |if (!$found) continue;
-         |$numOutput.add(1);
-         |${consume(ctx, input)}
-       """.stripMargin
-    }
-  }
-
-  /**
-   * Generates the code for anti join.
-   */
-  private def codegenAnti(ctx: CodegenContext, input: Seq[ExprCode]): String = {
-    val (broadcastRelation, relationTerm) = prepareBroadcast(ctx)
-    val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input)
-    val (matched, checkCondition, _) = getJoinCondition(ctx, input)
-    val numOutput = metricTerm(ctx, "numOutputRows")
-
-    if (broadcastRelation.value.keyIsUnique) {
-      s"""
-         |// generate join key for stream side
-         |${keyEv.code}
-         |// Check if the key has nulls.
-         |if (!($anyNull)) {
-         |  // Check if the HashedRelation exists.
-         |  UnsafeRow $matched = (UnsafeRow)$relationTerm.getValue(${keyEv.value});
-         |  if ($matched != null) {
-         |    // Evaluate the condition.
-         |    $checkCondition
-         |  }
-         |}
-         |$numOutput.add(1);
-         |${consume(ctx, input)}
-       """.stripMargin
-    } else {
-      val matches = ctx.freshName("matches")
-      val iteratorCls = classOf[Iterator[UnsafeRow]].getName
-      val found = ctx.freshName("found")
-      s"""
-         |// generate join key for stream side
-         |${keyEv.code}
-         |// Check if the key has nulls.
-         |if (!($anyNull)) {
-         |  // Check if the HashedRelation exists.
-         |  $iteratorCls $matches = ($iteratorCls)$relationTerm.get(${keyEv.value});
-         |  if ($matches != null) {
-         |    // Evaluate the condition.
-         |    boolean $found = false;
-         |    while (!$found && $matches.hasNext()) {
-         |      UnsafeRow $matched = (UnsafeRow) $matches.next();
-         |      $checkCondition
-         |      $found = true;
-         |    }
-         |    if ($found) continue;
-         |  }
-         |}
-         |$numOutput.add(1);
-         |${consume(ctx, input)}
-       """.stripMargin
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
new file mode 100644
index 0000000..51399e1
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.joins
+
+import org.apache.spark.TaskContext
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.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.{BinaryExecNode, CodegenSupport, SparkPlan}
+import org.apache.spark.sql.execution.metric.SQLMetrics
+import org.apache.spark.sql.types.LongType
+
+/**
+ * Performs an inner hash join of two child relations.  When the output RDD of this operator is
+ * being constructed, a Spark job is asynchronously started to calculate the values for the
+ * broadcast relation.  This data is then placed in a Spark broadcast variable.  The streamed
+ * relation is not shuffled.
+ */
+case class BroadcastHashJoinExec(
+    leftKeys: Seq[Expression],
+    rightKeys: Seq[Expression],
+    joinType: JoinType,
+    buildSide: BuildSide,
+    condition: Option[Expression],
+    left: SparkPlan,
+    right: SparkPlan)
+  extends BinaryExecNode with HashJoin with CodegenSupport {
+
+  override private[sql] lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
+
+  override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning
+
+  override def requiredChildDistribution: Seq[Distribution] = {
+    val mode = HashedRelationBroadcastMode(buildKeys)
+    buildSide match {
+      case BuildLeft =>
+        BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil
+      case BuildRight =>
+        UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil
+    }
+  }
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    val numOutputRows = longMetric("numOutputRows")
+
+    val broadcastRelation = buildPlan.executeBroadcast[HashedRelation]()
+    streamedPlan.execute().mapPartitions { streamedIter =>
+      val hashed = broadcastRelation.value.asReadOnlyCopy()
+      TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize)
+      join(streamedIter, hashed, numOutputRows)
+    }
+  }
+
+  override def inputRDDs(): Seq[RDD[InternalRow]] = {
+    streamedPlan.asInstanceOf[CodegenSupport].inputRDDs()
+  }
+
+  override def doProduce(ctx: CodegenContext): String = {
+    streamedPlan.asInstanceOf[CodegenSupport].produce(ctx, this)
+  }
+
+  override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
+    joinType match {
+      case Inner => codegenInner(ctx, input)
+      case LeftOuter | RightOuter => codegenOuter(ctx, input)
+      case LeftSemi => codegenSemi(ctx, input)
+      case LeftAnti => codegenAnti(ctx, input)
+      case x =>
+        throw new IllegalArgumentException(
+          s"BroadcastHashJoin should not take $x as the JoinType")
+    }
+  }
+
+  /**
+   * Returns a tuple of Broadcast of HashedRelation and the variable name for it.
+   */
+  private def prepareBroadcast(ctx: CodegenContext): (Broadcast[HashedRelation], String) = {
+    // create a name for HashedRelation
+    val broadcastRelation = buildPlan.executeBroadcast[HashedRelation]()
+    val broadcast = ctx.addReferenceObj("broadcast", broadcastRelation)
+    val relationTerm = ctx.freshName("relation")
+    val clsName = broadcastRelation.value.getClass.getName
+    ctx.addMutableState(clsName, relationTerm,
+      s"""
+         | $relationTerm = (($clsName) $broadcast.value()).asReadOnlyCopy();
+         | incPeakExecutionMemory($relationTerm.estimatedSize());
+       """.stripMargin)
+    (broadcastRelation, relationTerm)
+  }
+
+  /**
+   * Returns the code for generating join key for stream side, and expression of whether the key
+   * has any null in it or not.
+   */
+  private def genStreamSideJoinKey(
+      ctx: CodegenContext,
+      input: Seq[ExprCode]): (ExprCode, String) = {
+    ctx.currentVars = input
+    if (streamedKeys.length == 1 && streamedKeys.head.dataType == LongType) {
+      // generate the join key as Long
+      val ev = streamedKeys.head.genCode(ctx)
+      (ev, ev.isNull)
+    } else {
+      // generate the join key as UnsafeRow
+      val ev = GenerateUnsafeProjection.createCode(ctx, streamedKeys)
+      (ev, s"${ev.value}.anyNull()")
+    }
+  }
+
+  /**
+   * Generates the code for variable of build side.
+   */
+  private def genBuildSideVars(ctx: CodegenContext, matched: String): Seq[ExprCode] = {
+    ctx.currentVars = null
+    ctx.INPUT_ROW = matched
+    buildPlan.output.zipWithIndex.map { case (a, i) =>
+      val ev = BoundReference(i, a.dataType, a.nullable).genCode(ctx)
+      if (joinType == Inner) {
+        ev
+      } else {
+        // the variables are needed even there is no matched rows
+        val isNull = ctx.freshName("isNull")
+        val value = ctx.freshName("value")
+        val code = s"""
+          |boolean $isNull = true;
+          |${ctx.javaType(a.dataType)} $value = ${ctx.defaultValue(a.dataType)};
+          |if ($matched != null) {
+          |  ${ev.code}
+          |  $isNull = ${ev.isNull};
+          |  $value = ${ev.value};
+          |}
+         """.stripMargin
+        ExprCode(code, isNull, value)
+      }
+    }
+  }
+
+  /**
+   * Generate the (non-equi) condition used to filter joined rows. This is used in Inner, Left Semi
+   * and Left Anti joins.
+   */
+  private def getJoinCondition(
+      ctx: CodegenContext,
+      input: Seq[ExprCode]): (String, String, Seq[ExprCode]) = {
+    val matched = ctx.freshName("matched")
+    val buildVars = genBuildSideVars(ctx, matched)
+    val checkCondition = if (condition.isDefined) {
+      val expr = condition.get
+      // evaluate the variables from build side that used by condition
+      val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references)
+      // filter the output via condition
+      ctx.currentVars = input ++ buildVars
+      val ev =
+        BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx)
+      s"""
+         |$eval
+         |${ev.code}
+         |if (${ev.isNull} || !${ev.value}) continue;
+       """.stripMargin
+    } else {
+      ""
+    }
+    (matched, checkCondition, buildVars)
+  }
+
+  /**
+   * Generates the code for Inner join.
+   */
+  private def codegenInner(ctx: CodegenContext, input: Seq[ExprCode]): String = {
+    val (broadcastRelation, relationTerm) = prepareBroadcast(ctx)
+    val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input)
+    val (matched, checkCondition, buildVars) = getJoinCondition(ctx, input)
+    val numOutput = metricTerm(ctx, "numOutputRows")
+
+    val resultVars = buildSide match {
+      case BuildLeft => buildVars ++ input
+      case BuildRight => input ++ buildVars
+    }
+    if (broadcastRelation.value.keyIsUnique) {
+      s"""
+         |// generate join key for stream side
+         |${keyEv.code}
+         |// find matches from HashedRelation
+         |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value});
+         |if ($matched == null) continue;
+         |$checkCondition
+         |$numOutput.add(1);
+         |${consume(ctx, resultVars)}
+       """.stripMargin
+
+    } else {
+      ctx.copyResult = true
+      val matches = ctx.freshName("matches")
+      val iteratorCls = classOf[Iterator[UnsafeRow]].getName
+      s"""
+         |// generate join key for stream side
+         |${keyEv.code}
+         |// find matches from HashRelation
+         |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value});
+         |if ($matches == null) continue;
+         |while ($matches.hasNext()) {
+         |  UnsafeRow $matched = (UnsafeRow) $matches.next();
+         |  $checkCondition
+         |  $numOutput.add(1);
+         |  ${consume(ctx, resultVars)}
+         |}
+       """.stripMargin
+    }
+  }
+
+  /**
+   * Generates the code for left or right outer join.
+   */
+  private def codegenOuter(ctx: CodegenContext, input: Seq[ExprCode]): String = {
+    val (broadcastRelation, relationTerm) = prepareBroadcast(ctx)
+    val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input)
+    val matched = ctx.freshName("matched")
+    val buildVars = genBuildSideVars(ctx, matched)
+    val numOutput = metricTerm(ctx, "numOutputRows")
+
+    // filter the output via condition
+    val conditionPassed = ctx.freshName("conditionPassed")
+    val checkCondition = if (condition.isDefined) {
+      val expr = condition.get
+      // evaluate the variables from build side that used by condition
+      val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references)
+      ctx.currentVars = input ++ buildVars
+      val ev =
+        BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx)
+      s"""
+         |boolean $conditionPassed = true;
+         |${eval.trim}
+         |${ev.code}
+         |if ($matched != null) {
+         |  $conditionPassed = !${ev.isNull} && ${ev.value};
+         |}
+       """.stripMargin
+    } else {
+      s"final boolean $conditionPassed = true;"
+    }
+
+    val resultVars = buildSide match {
+      case BuildLeft => buildVars ++ input
+      case BuildRight => input ++ buildVars
+    }
+    if (broadcastRelation.value.keyIsUnique) {
+      s"""
+         |// generate join key for stream side
+         |${keyEv.code}
+         |// find matches from HashedRelation
+         |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value});
+         |${checkCondition.trim}
+         |if (!$conditionPassed) {
+         |  $matched = null;
+         |  // reset the variables those are already evaluated.
+         |  ${buildVars.filter(_.code == "").map(v => s"${v.isNull} = true;").mkString("\n")}
+         |}
+         |$numOutput.add(1);
+         |${consume(ctx, resultVars)}
+       """.stripMargin
+
+    } else {
+      ctx.copyResult = true
+      val matches = ctx.freshName("matches")
+      val iteratorCls = classOf[Iterator[UnsafeRow]].getName
+      val found = ctx.freshName("found")
+      s"""
+         |// generate join key for stream side
+         |${keyEv.code}
+         |// find matches from HashRelation
+         |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value});
+         |boolean $found = false;
+         |// the last iteration of this loop is to emit an empty row if there is no matched rows.
+         |while ($matches != null && $matches.hasNext() || !$found) {
+         |  UnsafeRow $matched = $matches != null && $matches.hasNext() ?
+         |    (UnsafeRow) $matches.next() : null;
+         |  ${checkCondition.trim}
+         |  if (!$conditionPassed) continue;
+         |  $found = true;
+         |  $numOutput.add(1);
+         |  ${consume(ctx, resultVars)}
+         |}
+       """.stripMargin
+    }
+  }
+
+  /**
+   * Generates the code for left semi join.
+   */
+  private def codegenSemi(ctx: CodegenContext, input: Seq[ExprCode]): String = {
+    val (broadcastRelation, relationTerm) = prepareBroadcast(ctx)
+    val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input)
+    val (matched, checkCondition, _) = getJoinCondition(ctx, input)
+    val numOutput = metricTerm(ctx, "numOutputRows")
+    if (broadcastRelation.value.keyIsUnique) {
+      s"""
+         |// generate join key for stream side
+         |${keyEv.code}
+         |// find matches from HashedRelation
+         |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value});
+         |if ($matched == null) continue;
+         |$checkCondition
+         |$numOutput.add(1);
+         |${consume(ctx, input)}
+       """.stripMargin
+    } else {
+      val matches = ctx.freshName("matches")
+      val iteratorCls = classOf[Iterator[UnsafeRow]].getName
+      val found = ctx.freshName("found")
+      s"""
+         |// generate join key for stream side
+         |${keyEv.code}
+         |// find matches from HashRelation
+         |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value});
+         |if ($matches == null) continue;
+         |boolean $found = false;
+         |while (!$found && $matches.hasNext()) {
+         |  UnsafeRow $matched = (UnsafeRow) $matches.next();
+         |  $checkCondition
+         |  $found = true;
+         |}
+         |if (!$found) continue;
+         |$numOutput.add(1);
+         |${consume(ctx, input)}
+       """.stripMargin
+    }
+  }
+
+  /**
+   * Generates the code for anti join.
+   */
+  private def codegenAnti(ctx: CodegenContext, input: Seq[ExprCode]): String = {
+    val (broadcastRelation, relationTerm) = prepareBroadcast(ctx)
+    val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input)
+    val (matched, checkCondition, _) = getJoinCondition(ctx, input)
+    val numOutput = metricTerm(ctx, "numOutputRows")
+
+    if (broadcastRelation.value.keyIsUnique) {
+      s"""
+         |// generate join key for stream side
+         |${keyEv.code}
+         |// Check if the key has nulls.
+         |if (!($anyNull)) {
+         |  // Check if the HashedRelation exists.
+         |  UnsafeRow $matched = (UnsafeRow)$relationTerm.getValue(${keyEv.value});
+         |  if ($matched != null) {
+         |    // Evaluate the condition.
+         |    $checkCondition
+         |  }
+         |}
+         |$numOutput.add(1);
+         |${consume(ctx, input)}
+       """.stripMargin
+    } else {
+      val matches = ctx.freshName("matches")
+      val iteratorCls = classOf[Iterator[UnsafeRow]].getName
+      val found = ctx.freshName("found")
+      s"""
+         |// generate join key for stream side
+         |${keyEv.code}
+         |// Check if the key has nulls.
+         |if (!($anyNull)) {
+         |  // Check if the HashedRelation exists.
+         |  $iteratorCls $matches = ($iteratorCls)$relationTerm.get(${keyEv.value});
+         |  if ($matches != null) {
+         |    // Evaluate the condition.
+         |    boolean $found = false;
+         |    while (!$found && $matches.hasNext()) {
+         |      UnsafeRow $matched = (UnsafeRow) $matches.next();
+         |      $checkCondition
+         |      $found = true;
+         |    }
+         |    if ($found) continue;
+         |  }
+         |}
+         |$numOutput.add(1);
+         |${consume(ctx, input)}
+       """.stripMargin
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala
----------------------------------------------------------------------
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/BroadcastNestedLoopJoin.scala
deleted file mode 100644
index 4ba710c..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala
+++ /dev/null
@@ -1,331 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.joins
-
-import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
-import org.apache.spark.sql.execution.metric.SQLMetrics
-import org.apache.spark.util.collection.{BitSet, CompactBuffer}
-
-case class BroadcastNestedLoopJoin(
-    left: SparkPlan,
-    right: SparkPlan,
-    buildSide: BuildSide,
-    joinType: JoinType,
-    condition: Option[Expression]) extends BinaryNode {
-
-  override private[sql] lazy val metrics = Map(
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
-
-  /** BuildRight means the right relation <=> the broadcast relation. */
-  private val (streamed, broadcast) = buildSide match {
-    case BuildRight => (left, right)
-    case BuildLeft => (right, left)
-  }
-
-  override def requiredChildDistribution: Seq[Distribution] = buildSide match {
-    case BuildLeft =>
-      BroadcastDistribution(IdentityBroadcastMode) :: UnspecifiedDistribution :: Nil
-    case BuildRight =>
-      UnspecifiedDistribution :: BroadcastDistribution(IdentityBroadcastMode) :: Nil
-  }
-
-  private[this] def genResultProjection: InternalRow => InternalRow = {
-    if (joinType == LeftSemi) {
-      UnsafeProjection.create(output, output)
-    } else {
-      // Always put the stream side on left to simplify implementation
-      // both of left and right side could be null
-      UnsafeProjection.create(
-        output, (streamed.output ++ broadcast.output).map(_.withNullability(true)))
-    }
-  }
-
-  override def outputPartitioning: Partitioning = streamed.outputPartitioning
-
-  override def output: Seq[Attribute] = {
-    joinType match {
-      case Inner =>
-        left.output ++ right.output
-      case LeftOuter =>
-        left.output ++ right.output.map(_.withNullability(true))
-      case RightOuter =>
-        left.output.map(_.withNullability(true)) ++ right.output
-      case FullOuter =>
-        left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true))
-      case LeftExistence(_) =>
-        left.output
-      case x =>
-        throw new IllegalArgumentException(
-          s"BroadcastNestedLoopJoin should not take $x as the JoinType")
-    }
-  }
-
-  @transient private lazy val boundCondition = {
-    if (condition.isDefined) {
-      newPredicate(condition.get, streamed.output ++ broadcast.output)
-    } else {
-      (r: InternalRow) => true
-    }
-  }
-
-  /**
-   * The implementation for InnerJoin.
-   */
-  private def innerJoin(relation: Broadcast[Array[InternalRow]]): RDD[InternalRow] = {
-    streamed.execute().mapPartitionsInternal { streamedIter =>
-      val buildRows = relation.value
-      val joinedRow = new JoinedRow
-
-      streamedIter.flatMap { streamedRow =>
-        val joinedRows = buildRows.iterator.map(r => joinedRow(streamedRow, r))
-        if (condition.isDefined) {
-          joinedRows.filter(boundCondition)
-        } else {
-          joinedRows
-        }
-      }
-    }
-  }
-
-  /**
-   * The implementation for these joins:
-   *
-   *   LeftOuter with BuildRight
-   *   RightOuter with BuildLeft
-   */
-  private def outerJoin(relation: Broadcast[Array[InternalRow]]): RDD[InternalRow] = {
-    streamed.execute().mapPartitionsInternal { streamedIter =>
-      val buildRows = relation.value
-      val joinedRow = new JoinedRow
-      val nulls = new GenericMutableRow(broadcast.output.size)
-
-      // Returns an iterator to avoid copy the rows.
-      new Iterator[InternalRow] {
-        // current row from stream side
-        private var streamRow: InternalRow = null
-        // have found a match for current row or not
-        private var foundMatch: Boolean = false
-        // the matched result row
-        private var resultRow: InternalRow = null
-        // the next index of buildRows to try
-        private var nextIndex: Int = 0
-
-        private def findNextMatch(): Boolean = {
-          if (streamRow == null) {
-            if (!streamedIter.hasNext) {
-              return false
-            }
-            streamRow = streamedIter.next()
-            nextIndex = 0
-            foundMatch = false
-          }
-          while (nextIndex < buildRows.length) {
-            resultRow = joinedRow(streamRow, buildRows(nextIndex))
-            nextIndex += 1
-            if (boundCondition(resultRow)) {
-              foundMatch = true
-              return true
-            }
-          }
-          if (!foundMatch) {
-            resultRow = joinedRow(streamRow, nulls)
-            streamRow = null
-            true
-          } else {
-            resultRow = null
-            streamRow = null
-            findNextMatch()
-          }
-        }
-
-        override def hasNext(): Boolean = {
-          resultRow != null || findNextMatch()
-        }
-        override def next(): InternalRow = {
-          val r = resultRow
-          resultRow = null
-          r
-        }
-      }
-    }
-  }
-
-  /**
-   * The implementation for these joins:
-   *
-   *   LeftSemi with BuildRight
-   *   Anti with BuildRight
-   */
-  private def leftExistenceJoin(
-      relation: Broadcast[Array[InternalRow]],
-      exists: Boolean): RDD[InternalRow] = {
-    assert(buildSide == BuildRight)
-    streamed.execute().mapPartitionsInternal { streamedIter =>
-      val buildRows = relation.value
-      val joinedRow = new JoinedRow
-
-      if (condition.isDefined) {
-        streamedIter.filter(l =>
-          buildRows.exists(r => boundCondition(joinedRow(l, r))) == exists
-        )
-      } else if (buildRows.nonEmpty == exists) {
-        streamedIter
-      } else {
-        Iterator.empty
-      }
-    }
-  }
-
-  /**
-   * The implementation for these joins:
-   *
-   *   LeftOuter with BuildLeft
-   *   RightOuter with BuildRight
-   *   FullOuter
-   *   LeftSemi with BuildLeft
-   *   Anti with BuildLeft
-   */
-  private def defaultJoin(relation: Broadcast[Array[InternalRow]]): RDD[InternalRow] = {
-    /** All rows that either match both-way, or rows from streamed joined with nulls. */
-    val streamRdd = streamed.execute()
-
-    val matchedBuildRows = streamRdd.mapPartitionsInternal { streamedIter =>
-      val buildRows = relation.value
-      val matched = new BitSet(buildRows.length)
-      val joinedRow = new JoinedRow
-
-      streamedIter.foreach { streamedRow =>
-        var i = 0
-        while (i < buildRows.length) {
-          if (boundCondition(joinedRow(streamedRow, buildRows(i)))) {
-            matched.set(i)
-          }
-          i += 1
-        }
-      }
-      Seq(matched).toIterator
-    }
-
-    val matchedBroadcastRows = matchedBuildRows.fold(
-      new BitSet(relation.value.length)
-    )(_ | _)
-
-    if (joinType == LeftSemi) {
-      assert(buildSide == BuildLeft)
-      val buf: CompactBuffer[InternalRow] = new CompactBuffer()
-      var i = 0
-      val rel = relation.value
-      while (i < rel.length) {
-        if (matchedBroadcastRows.get(i)) {
-          buf += rel(i).copy()
-        }
-        i += 1
-      }
-      return sparkContext.makeRDD(buf)
-    }
-
-    val notMatchedBroadcastRows: Seq[InternalRow] = {
-      val nulls = new GenericMutableRow(streamed.output.size)
-      val buf: CompactBuffer[InternalRow] = new CompactBuffer()
-      var i = 0
-      val buildRows = relation.value
-      val joinedRow = new JoinedRow
-      joinedRow.withLeft(nulls)
-      while (i < buildRows.length) {
-        if (!matchedBroadcastRows.get(i)) {
-          buf += joinedRow.withRight(buildRows(i)).copy()
-        }
-        i += 1
-      }
-      buf
-    }
-
-    if (joinType == LeftAnti) {
-      return sparkContext.makeRDD(notMatchedBroadcastRows)
-    }
-
-    val matchedStreamRows = streamRdd.mapPartitionsInternal { streamedIter =>
-      val buildRows = relation.value
-      val joinedRow = new JoinedRow
-      val nulls = new GenericMutableRow(broadcast.output.size)
-
-      streamedIter.flatMap { streamedRow =>
-        var i = 0
-        var foundMatch = false
-        val matchedRows = new CompactBuffer[InternalRow]
-
-        while (i < buildRows.length) {
-          if (boundCondition(joinedRow(streamedRow, buildRows(i)))) {
-            matchedRows += joinedRow.copy()
-            foundMatch = true
-          }
-          i += 1
-        }
-
-        if (!foundMatch && joinType == FullOuter) {
-          matchedRows += joinedRow(streamedRow, nulls).copy()
-        }
-        matchedRows.iterator
-      }
-    }
-
-    sparkContext.union(
-      matchedStreamRows,
-      sparkContext.makeRDD(notMatchedBroadcastRows)
-    )
-  }
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    val broadcastedRelation = broadcast.executeBroadcast[Array[InternalRow]]()
-
-    val resultRdd = (joinType, buildSide) match {
-      case (Inner, _) =>
-        innerJoin(broadcastedRelation)
-      case (LeftOuter, BuildRight) | (RightOuter, BuildLeft) =>
-        outerJoin(broadcastedRelation)
-      case (LeftSemi, BuildRight) =>
-        leftExistenceJoin(broadcastedRelation, exists = true)
-      case (LeftAnti, BuildRight) =>
-        leftExistenceJoin(broadcastedRelation, exists = false)
-      case _ =>
-        /**
-         * LeftOuter with BuildLeft
-         * RightOuter with BuildRight
-         * FullOuter
-         * LeftSemi with BuildLeft
-         * Anti with BuildLeft
-         */
-        defaultJoin(broadcastedRelation)
-    }
-
-    val numOutputRows = longMetric("numOutputRows")
-    resultRdd.mapPartitionsInternal { iter =>
-      val resultProj = genResultProjection
-      iter.map { r =>
-        numOutputRows += 1
-        resultProj(r)
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala
new file mode 100644
index 0000000..51afa00
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.joins
+
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.physical._
+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 BroadcastNestedLoopJoinExec(
+    left: SparkPlan,
+    right: SparkPlan,
+    buildSide: BuildSide,
+    joinType: JoinType,
+    condition: Option[Expression]) extends BinaryExecNode {
+
+  override private[sql] lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
+
+  /** BuildRight means the right relation <=> the broadcast relation. */
+  private val (streamed, broadcast) = buildSide match {
+    case BuildRight => (left, right)
+    case BuildLeft => (right, left)
+  }
+
+  override def requiredChildDistribution: Seq[Distribution] = buildSide match {
+    case BuildLeft =>
+      BroadcastDistribution(IdentityBroadcastMode) :: UnspecifiedDistribution :: Nil
+    case BuildRight =>
+      UnspecifiedDistribution :: BroadcastDistribution(IdentityBroadcastMode) :: Nil
+  }
+
+  private[this] def genResultProjection: InternalRow => InternalRow = {
+    if (joinType == LeftSemi) {
+      UnsafeProjection.create(output, output)
+    } else {
+      // Always put the stream side on left to simplify implementation
+      // both of left and right side could be null
+      UnsafeProjection.create(
+        output, (streamed.output ++ broadcast.output).map(_.withNullability(true)))
+    }
+  }
+
+  override def outputPartitioning: Partitioning = streamed.outputPartitioning
+
+  override def output: Seq[Attribute] = {
+    joinType match {
+      case Inner =>
+        left.output ++ right.output
+      case LeftOuter =>
+        left.output ++ right.output.map(_.withNullability(true))
+      case RightOuter =>
+        left.output.map(_.withNullability(true)) ++ right.output
+      case FullOuter =>
+        left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true))
+      case LeftExistence(_) =>
+        left.output
+      case x =>
+        throw new IllegalArgumentException(
+          s"BroadcastNestedLoopJoin should not take $x as the JoinType")
+    }
+  }
+
+  @transient private lazy val boundCondition = {
+    if (condition.isDefined) {
+      newPredicate(condition.get, streamed.output ++ broadcast.output)
+    } else {
+      (r: InternalRow) => true
+    }
+  }
+
+  /**
+   * The implementation for InnerJoin.
+   */
+  private def innerJoin(relation: Broadcast[Array[InternalRow]]): RDD[InternalRow] = {
+    streamed.execute().mapPartitionsInternal { streamedIter =>
+      val buildRows = relation.value
+      val joinedRow = new JoinedRow
+
+      streamedIter.flatMap { streamedRow =>
+        val joinedRows = buildRows.iterator.map(r => joinedRow(streamedRow, r))
+        if (condition.isDefined) {
+          joinedRows.filter(boundCondition)
+        } else {
+          joinedRows
+        }
+      }
+    }
+  }
+
+  /**
+   * The implementation for these joins:
+   *
+   *   LeftOuter with BuildRight
+   *   RightOuter with BuildLeft
+   */
+  private def outerJoin(relation: Broadcast[Array[InternalRow]]): RDD[InternalRow] = {
+    streamed.execute().mapPartitionsInternal { streamedIter =>
+      val buildRows = relation.value
+      val joinedRow = new JoinedRow
+      val nulls = new GenericMutableRow(broadcast.output.size)
+
+      // Returns an iterator to avoid copy the rows.
+      new Iterator[InternalRow] {
+        // current row from stream side
+        private var streamRow: InternalRow = null
+        // have found a match for current row or not
+        private var foundMatch: Boolean = false
+        // the matched result row
+        private var resultRow: InternalRow = null
+        // the next index of buildRows to try
+        private var nextIndex: Int = 0
+
+        private def findNextMatch(): Boolean = {
+          if (streamRow == null) {
+            if (!streamedIter.hasNext) {
+              return false
+            }
+            streamRow = streamedIter.next()
+            nextIndex = 0
+            foundMatch = false
+          }
+          while (nextIndex < buildRows.length) {
+            resultRow = joinedRow(streamRow, buildRows(nextIndex))
+            nextIndex += 1
+            if (boundCondition(resultRow)) {
+              foundMatch = true
+              return true
+            }
+          }
+          if (!foundMatch) {
+            resultRow = joinedRow(streamRow, nulls)
+            streamRow = null
+            true
+          } else {
+            resultRow = null
+            streamRow = null
+            findNextMatch()
+          }
+        }
+
+        override def hasNext(): Boolean = {
+          resultRow != null || findNextMatch()
+        }
+        override def next(): InternalRow = {
+          val r = resultRow
+          resultRow = null
+          r
+        }
+      }
+    }
+  }
+
+  /**
+   * The implementation for these joins:
+   *
+   *   LeftSemi with BuildRight
+   *   Anti with BuildRight
+   */
+  private def leftExistenceJoin(
+      relation: Broadcast[Array[InternalRow]],
+      exists: Boolean): RDD[InternalRow] = {
+    assert(buildSide == BuildRight)
+    streamed.execute().mapPartitionsInternal { streamedIter =>
+      val buildRows = relation.value
+      val joinedRow = new JoinedRow
+
+      if (condition.isDefined) {
+        streamedIter.filter(l =>
+          buildRows.exists(r => boundCondition(joinedRow(l, r))) == exists
+        )
+      } else if (buildRows.nonEmpty == exists) {
+        streamedIter
+      } else {
+        Iterator.empty
+      }
+    }
+  }
+
+  /**
+   * The implementation for these joins:
+   *
+   *   LeftOuter with BuildLeft
+   *   RightOuter with BuildRight
+   *   FullOuter
+   *   LeftSemi with BuildLeft
+   *   Anti with BuildLeft
+   */
+  private def defaultJoin(relation: Broadcast[Array[InternalRow]]): RDD[InternalRow] = {
+    /** All rows that either match both-way, or rows from streamed joined with nulls. */
+    val streamRdd = streamed.execute()
+
+    val matchedBuildRows = streamRdd.mapPartitionsInternal { streamedIter =>
+      val buildRows = relation.value
+      val matched = new BitSet(buildRows.length)
+      val joinedRow = new JoinedRow
+
+      streamedIter.foreach { streamedRow =>
+        var i = 0
+        while (i < buildRows.length) {
+          if (boundCondition(joinedRow(streamedRow, buildRows(i)))) {
+            matched.set(i)
+          }
+          i += 1
+        }
+      }
+      Seq(matched).toIterator
+    }
+
+    val matchedBroadcastRows = matchedBuildRows.fold(
+      new BitSet(relation.value.length)
+    )(_ | _)
+
+    if (joinType == LeftSemi) {
+      assert(buildSide == BuildLeft)
+      val buf: CompactBuffer[InternalRow] = new CompactBuffer()
+      var i = 0
+      val rel = relation.value
+      while (i < rel.length) {
+        if (matchedBroadcastRows.get(i)) {
+          buf += rel(i).copy()
+        }
+        i += 1
+      }
+      return sparkContext.makeRDD(buf)
+    }
+
+    val notMatchedBroadcastRows: Seq[InternalRow] = {
+      val nulls = new GenericMutableRow(streamed.output.size)
+      val buf: CompactBuffer[InternalRow] = new CompactBuffer()
+      var i = 0
+      val buildRows = relation.value
+      val joinedRow = new JoinedRow
+      joinedRow.withLeft(nulls)
+      while (i < buildRows.length) {
+        if (!matchedBroadcastRows.get(i)) {
+          buf += joinedRow.withRight(buildRows(i)).copy()
+        }
+        i += 1
+      }
+      buf
+    }
+
+    if (joinType == LeftAnti) {
+      return sparkContext.makeRDD(notMatchedBroadcastRows)
+    }
+
+    val matchedStreamRows = streamRdd.mapPartitionsInternal { streamedIter =>
+      val buildRows = relation.value
+      val joinedRow = new JoinedRow
+      val nulls = new GenericMutableRow(broadcast.output.size)
+
+      streamedIter.flatMap { streamedRow =>
+        var i = 0
+        var foundMatch = false
+        val matchedRows = new CompactBuffer[InternalRow]
+
+        while (i < buildRows.length) {
+          if (boundCondition(joinedRow(streamedRow, buildRows(i)))) {
+            matchedRows += joinedRow.copy()
+            foundMatch = true
+          }
+          i += 1
+        }
+
+        if (!foundMatch && joinType == FullOuter) {
+          matchedRows += joinedRow(streamedRow, nulls).copy()
+        }
+        matchedRows.iterator
+      }
+    }
+
+    sparkContext.union(
+      matchedStreamRows,
+      sparkContext.makeRDD(notMatchedBroadcastRows)
+    )
+  }
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    val broadcastedRelation = broadcast.executeBroadcast[Array[InternalRow]]()
+
+    val resultRdd = (joinType, buildSide) match {
+      case (Inner, _) =>
+        innerJoin(broadcastedRelation)
+      case (LeftOuter, BuildRight) | (RightOuter, BuildLeft) =>
+        outerJoin(broadcastedRelation)
+      case (LeftSemi, BuildRight) =>
+        leftExistenceJoin(broadcastedRelation, exists = true)
+      case (LeftAnti, BuildRight) =>
+        leftExistenceJoin(broadcastedRelation, exists = false)
+      case _ =>
+        /**
+         * LeftOuter with BuildLeft
+         * RightOuter with BuildRight
+         * FullOuter
+         * LeftSemi with BuildLeft
+         * Anti with BuildLeft
+         */
+        defaultJoin(broadcastedRelation)
+    }
+
+    val numOutputRows = longMetric("numOutputRows")
+    resultRdd.mapPartitionsInternal { iter =>
+      val resultProj = genResultProjection
+      iter.map { r =>
+        numOutputRows += 1
+        resultProj(r)
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala
----------------------------------------------------------------------
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/CartesianProduct.scala
deleted file mode 100644
index b1de52b..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.joins
-
-import org.apache.spark._
-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.metric.SQLMetrics
-import org.apache.spark.util.CompletionIterator
-import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter
-
-/**
- * An optimized CartesianRDD for UnsafeRow, which will cache the rows from second child RDD,
- * will be much faster than building the right partition for every row in left RDD, it also
- * materialize the right RDD (in case of the right RDD is nondeterministic).
- */
-private[spark]
-class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numFieldsOfRight: Int)
-  extends CartesianRDD[UnsafeRow, UnsafeRow](left.sparkContext, left, right) {
-
-  override def compute(split: Partition, context: TaskContext): Iterator[(UnsafeRow, UnsafeRow)] = {
-    // We will not sort the rows, so prefixComparator and recordComparator are null.
-    val sorter = UnsafeExternalSorter.create(
-      context.taskMemoryManager(),
-      SparkEnv.get.blockManager,
-      SparkEnv.get.serializerManager,
-      context,
-      null,
-      null,
-      1024,
-      SparkEnv.get.memoryManager.pageSizeBytes,
-      false)
-
-    val partition = split.asInstanceOf[CartesianPartition]
-    for (y <- rdd2.iterator(partition.s2, context)) {
-      sorter.insertRecord(y.getBaseObject, y.getBaseOffset, y.getSizeInBytes, 0)
-    }
-
-    // Create an iterator from sorter and wrapper it as Iterator[UnsafeRow]
-    def createIter(): Iterator[UnsafeRow] = {
-      val iter = sorter.getIterator
-      val unsafeRow = new UnsafeRow(numFieldsOfRight)
-      new Iterator[UnsafeRow] {
-        override def hasNext: Boolean = {
-          iter.hasNext
-        }
-        override def next(): UnsafeRow = {
-          iter.loadNext()
-          unsafeRow.pointTo(iter.getBaseObject, iter.getBaseOffset, iter.getRecordLength)
-          unsafeRow
-        }
-      }
-    }
-
-    val resultIter =
-      for (x <- rdd1.iterator(partition.s1, context);
-           y <- createIter()) yield (x, y)
-    CompletionIterator[(UnsafeRow, UnsafeRow), Iterator[(UnsafeRow, UnsafeRow)]](
-      resultIter, sorter.cleanupResources)
-  }
-}
-
-
-case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode {
-  override def output: Seq[Attribute] = left.output ++ right.output
-
-  override private[sql] lazy val metrics = Map(
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    val numOutputRows = longMetric("numOutputRows")
-
-    val leftResults = left.execute().asInstanceOf[RDD[UnsafeRow]]
-    val rightResults = right.execute().asInstanceOf[RDD[UnsafeRow]]
-
-    val pair = new UnsafeCartesianRDD(leftResults, rightResults, right.output.size)
-    pair.mapPartitionsInternal { iter =>
-      val joiner = GenerateUnsafeRowJoiner.create(left.schema, right.schema)
-      iter.map { r =>
-        numOutputRows += 1
-        joiner.join(r._1, r._2)
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala
new file mode 100644
index 0000000..3ce7c0e
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.joins
+
+import org.apache.spark._
+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.{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
+
+/**
+ * An optimized CartesianRDD for UnsafeRow, which will cache the rows from second child RDD,
+ * will be much faster than building the right partition for every row in left RDD, it also
+ * materialize the right RDD (in case of the right RDD is nondeterministic).
+ */
+private[spark]
+class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numFieldsOfRight: Int)
+  extends CartesianRDD[UnsafeRow, UnsafeRow](left.sparkContext, left, right) {
+
+  override def compute(split: Partition, context: TaskContext): Iterator[(UnsafeRow, UnsafeRow)] = {
+    // We will not sort the rows, so prefixComparator and recordComparator are null.
+    val sorter = UnsafeExternalSorter.create(
+      context.taskMemoryManager(),
+      SparkEnv.get.blockManager,
+      SparkEnv.get.serializerManager,
+      context,
+      null,
+      null,
+      1024,
+      SparkEnv.get.memoryManager.pageSizeBytes,
+      false)
+
+    val partition = split.asInstanceOf[CartesianPartition]
+    for (y <- rdd2.iterator(partition.s2, context)) {
+      sorter.insertRecord(y.getBaseObject, y.getBaseOffset, y.getSizeInBytes, 0)
+    }
+
+    // Create an iterator from sorter and wrapper it as Iterator[UnsafeRow]
+    def createIter(): Iterator[UnsafeRow] = {
+      val iter = sorter.getIterator
+      val unsafeRow = new UnsafeRow(numFieldsOfRight)
+      new Iterator[UnsafeRow] {
+        override def hasNext: Boolean = {
+          iter.hasNext
+        }
+        override def next(): UnsafeRow = {
+          iter.loadNext()
+          unsafeRow.pointTo(iter.getBaseObject, iter.getBaseOffset, iter.getRecordLength)
+          unsafeRow
+        }
+      }
+    }
+
+    val resultIter =
+      for (x <- rdd1.iterator(partition.s1, context);
+           y <- createIter()) yield (x, y)
+    CompletionIterator[(UnsafeRow, UnsafeRow), Iterator[(UnsafeRow, UnsafeRow)]](
+      resultIter, sorter.cleanupResources)
+  }
+}
+
+
+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(
+    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    val numOutputRows = longMetric("numOutputRows")
+
+    val leftResults = left.execute().asInstanceOf[RDD[UnsafeRow]]
+    val rightResults = right.execute().asInstanceOf[RDD[UnsafeRow]]
+
+    val pair = new UnsafeCartesianRDD(leftResults, rightResults, right.output.size)
+    pair.mapPartitionsInternal { iter =>
+      val joiner = GenerateUnsafeRowJoiner.create(left.schema, right.schema)
+      iter.map { r =>
+        numOutputRows += 1
+        joiner.join(r._1, r._2)
+      }
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[5/7] spark git commit: [SPARK-14855][SQL] Add "Exec" suffix to physical operators

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala
new file mode 100644
index 0000000..97bbab6
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala
@@ -0,0 +1,1008 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import java.util
+
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{SparkEnv, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.plans.physical._
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.util.collection.unsafe.sort.{UnsafeExternalSorter, UnsafeSorterIterator}
+
+/**
+ * This class calculates and outputs (windowed) aggregates over the rows in a single (sorted)
+ * partition. The aggregates are calculated for each row in the group. Special processing
+ * instructions, frames, are used to calculate these aggregates. Frames are processed in the order
+ * specified in the window specification (the ORDER BY ... clause). There are four different frame
+ * types:
+ * - Entire partition: The frame is the entire partition, i.e.
+ *   UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING. For this case, window function will take all
+ *   rows as inputs and be evaluated once.
+ * - Growing frame: We only add new rows into the frame, i.e. UNBOUNDED PRECEDING AND ....
+ *   Every time we move to a new row to process, we add some rows to the frame. We do not remove
+ *   rows from this frame.
+ * - Shrinking frame: We only remove rows from the frame, i.e. ... AND UNBOUNDED FOLLOWING.
+ *   Every time we move to a new row to process, we remove some rows from the frame. We do not add
+ *   rows to this frame.
+ * - Moving frame: Every time we move to a new row to process, we remove some rows from the frame
+ *   and we add some rows to the frame. Examples are:
+ *     1 PRECEDING AND CURRENT ROW and 1 FOLLOWING AND 2 FOLLOWING.
+ * - Offset frame: The frame consist of one row, which is an offset number of rows away from the
+ *   current row. Only [[OffsetWindowFunction]]s can be processed in an offset frame.
+ *
+ * Different frame boundaries can be used in Growing, Shrinking and Moving frames. A frame
+ * boundary can be either Row or Range based:
+ * - Row Based: A row based boundary is based on the position of the row within the partition.
+ *   An offset indicates the number of rows above or below the current row, the frame for the
+ *   current row starts or ends. For instance, given a row based sliding frame with a lower bound
+ *   offset of -1 and a upper bound offset of +2. The frame for row with index 5 would range from
+ *   index 4 to index 6.
+ * - Range based: A range based boundary is based on the actual value of the ORDER BY
+ *   expression(s). An offset is used to alter the value of the ORDER BY expression, for
+ *   instance if the current order by expression has a value of 10 and the lower bound offset
+ *   is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a
+ *   number of constraints on the ORDER BY expressions: there can be only one expression and this
+ *   expression must have a numerical data type. An exception can be made when the offset is 0,
+ *   because no value modification is needed, in this case multiple and non-numeric ORDER BY
+ *   expression are allowed.
+ *
+ * This is quite an expensive operator because every row for a single group must be in the same
+ * partition and partitions must be sorted according to the grouping and sort order. The operator
+ * requires the planner to take care of the partitioning and sorting.
+ *
+ * The operator is semi-blocking. The window functions and aggregates are calculated one group at
+ * a time, the result will only be made available after the processing for the entire group has
+ * finished. The operator is able to process different frame configurations at the same time. This
+ * is done by delegating the actual frame processing (i.e. calculation of the window functions) to
+ * specialized classes, see [[WindowFunctionFrame]], which take care of their own frame type:
+ * Entire Partition, Sliding, Growing & Shrinking. Boundary evaluation is also delegated to a pair
+ * of specialized classes: [[RowBoundOrdering]] & [[RangeBoundOrdering]].
+ */
+case class WindowExec(
+    windowExpression: Seq[NamedExpression],
+    partitionSpec: Seq[Expression],
+    orderSpec: Seq[SortOrder],
+    child: SparkPlan)
+  extends UnaryExecNode {
+
+  override def output: Seq[Attribute] =
+    child.output ++ windowExpression.map(_.toAttribute)
+
+  override def requiredChildDistribution: Seq[Distribution] = {
+    if (partitionSpec.isEmpty) {
+      // Only show warning when the number of bytes is larger than 100 MB?
+      logWarning("No Partition Defined for Window operation! Moving all data to a single "
+        + "partition, this can cause serious performance degradation.")
+      AllTuples :: Nil
+    } else ClusteredDistribution(partitionSpec) :: Nil
+  }
+
+  override def requiredChildOrdering: Seq[Seq[SortOrder]] =
+    Seq(partitionSpec.map(SortOrder(_, Ascending)) ++ orderSpec)
+
+  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
+
+  /**
+   * Create a bound ordering object for a given frame type and offset. A bound ordering object is
+   * used to determine which input row lies within the frame boundaries of an output row.
+   *
+   * This method uses Code Generation. It can only be used on the executor side.
+   *
+   * @param frameType to evaluate. This can either be Row or Range based.
+   * @param offset with respect to the row.
+   * @return a bound ordering object.
+   */
+  private[this] def createBoundOrdering(frameType: FrameType, offset: Int): BoundOrdering = {
+    frameType match {
+      case RangeFrame =>
+        val (exprs, current, bound) = if (offset == 0) {
+          // Use the entire order expression when the offset is 0.
+          val exprs = orderSpec.map(_.child)
+          val buildProjection = () => newMutableProjection(exprs, child.output)
+          (orderSpec, buildProjection(), buildProjection())
+        } else if (orderSpec.size == 1) {
+          // Use only the first order expression when the offset is non-null.
+          val sortExpr = orderSpec.head
+          val expr = sortExpr.child
+          // Create the projection which returns the current 'value'.
+          val current = newMutableProjection(expr :: Nil, child.output)
+          // Flip the sign of the offset when processing the order is descending
+          val boundOffset = sortExpr.direction match {
+            case Descending => -offset
+            case Ascending => offset
+          }
+          // Create the projection which returns the current 'value' modified by adding the offset.
+          val boundExpr = Add(expr, Cast(Literal.create(boundOffset, IntegerType), expr.dataType))
+          val bound = newMutableProjection(boundExpr :: Nil, child.output)
+          (sortExpr :: Nil, current, bound)
+        } else {
+          sys.error("Non-Zero range offsets are not supported for windows " +
+            "with multiple order expressions.")
+        }
+        // Construct the ordering. This is used to compare the result of current value projection
+        // to the result of bound value projection. This is done manually because we want to use
+        // Code Generation (if it is enabled).
+        val sortExprs = exprs.zipWithIndex.map { case (e, i) =>
+          SortOrder(BoundReference(i, e.dataType, e.nullable), e.direction)
+        }
+        val ordering = newOrdering(sortExprs, Nil)
+        RangeBoundOrdering(ordering, current, bound)
+      case RowFrame => RowBoundOrdering(offset)
+    }
+  }
+
+  /**
+   * Collection containing an entry for each window frame to process. Each entry contains a frames'
+   * WindowExpressions and factory function for the WindowFrameFunction.
+   */
+  private[this] lazy val windowFrameExpressionFactoryPairs = {
+    type FrameKey = (String, FrameType, Option[Int], Option[Int])
+    type ExpressionBuffer = mutable.Buffer[Expression]
+    val framedFunctions = mutable.Map.empty[FrameKey, (ExpressionBuffer, ExpressionBuffer)]
+
+    // Add a function and its function to the map for a given frame.
+    def collect(tpe: String, fr: SpecifiedWindowFrame, e: Expression, fn: Expression): Unit = {
+      val key = (tpe, fr.frameType, FrameBoundary(fr.frameStart), FrameBoundary(fr.frameEnd))
+      val (es, fns) = framedFunctions.getOrElseUpdate(
+        key, (ArrayBuffer.empty[Expression], ArrayBuffer.empty[Expression]))
+      es.append(e)
+      fns.append(fn)
+    }
+
+    // Collect all valid window functions and group them by their frame.
+    windowExpression.foreach { x =>
+      x.foreach {
+        case e @ WindowExpression(function, spec) =>
+          val frame = spec.frameSpecification.asInstanceOf[SpecifiedWindowFrame]
+          function match {
+            case AggregateExpression(f, _, _, _) => collect("AGGREGATE", frame, e, f)
+            case f: AggregateWindowFunction => collect("AGGREGATE", frame, e, f)
+            case f: OffsetWindowFunction => collect("OFFSET", frame, e, f)
+            case f => sys.error(s"Unsupported window function: $f")
+          }
+        case _ =>
+      }
+    }
+
+    // Map the groups to a (unbound) expression and frame factory pair.
+    var numExpressions = 0
+    framedFunctions.toSeq.map {
+      case (key, (expressions, functionSeq)) =>
+        val ordinal = numExpressions
+        val functions = functionSeq.toArray
+
+        // Construct an aggregate processor if we need one.
+        def processor = AggregateProcessor(
+          functions,
+          ordinal,
+          child.output,
+          (expressions, schema) =>
+            newMutableProjection(expressions, schema, subexpressionEliminationEnabled))
+
+        // Create the factory
+        val factory = key match {
+          // Offset Frame
+          case ("OFFSET", RowFrame, Some(offset), Some(h)) if offset == h =>
+            target: MutableRow =>
+              new OffsetWindowFunctionFrame(
+                target,
+                ordinal,
+                functions,
+                child.output,
+                (expressions, schema) =>
+                  newMutableProjection(expressions, schema, subexpressionEliminationEnabled),
+                offset)
+
+          // Growing Frame.
+          case ("AGGREGATE", frameType, None, Some(high)) =>
+            target: MutableRow => {
+              new UnboundedPrecedingWindowFunctionFrame(
+                target,
+                processor,
+                createBoundOrdering(frameType, high))
+            }
+
+          // Shrinking Frame.
+          case ("AGGREGATE", frameType, Some(low), None) =>
+            target: MutableRow => {
+              new UnboundedFollowingWindowFunctionFrame(
+                target,
+                processor,
+                createBoundOrdering(frameType, low))
+            }
+
+          // Moving Frame.
+          case ("AGGREGATE", frameType, Some(low), Some(high)) =>
+            target: MutableRow => {
+              new SlidingWindowFunctionFrame(
+                target,
+                processor,
+                createBoundOrdering(frameType, low),
+                createBoundOrdering(frameType, high))
+            }
+
+          // Entire Partition Frame.
+          case ("AGGREGATE", frameType, None, None) =>
+            target: MutableRow => {
+              new UnboundedWindowFunctionFrame(target, processor)
+            }
+        }
+
+        // Keep track of the number of expressions. This is a side-effect in a map...
+        numExpressions += expressions.size
+
+        // Create the Frame Expression - Factory pair.
+        (expressions, factory)
+    }
+  }
+
+  /**
+   * Create the resulting projection.
+   *
+   * This method uses Code Generation. It can only be used on the executor side.
+   *
+   * @param expressions unbound ordered function expressions.
+   * @return the final resulting projection.
+   */
+  private[this] def createResultProjection(
+      expressions: Seq[Expression]): UnsafeProjection = {
+    val references = expressions.zipWithIndex.map{ case (e, i) =>
+      // Results of window expressions will be on the right side of child's output
+      BoundReference(child.output.size + i, e.dataType, e.nullable)
+    }
+    val unboundToRefMap = expressions.zip(references).toMap
+    val patchedWindowExpression = windowExpression.map(_.transform(unboundToRefMap))
+    UnsafeProjection.create(
+      child.output ++ patchedWindowExpression,
+      child.output)
+  }
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    // Unwrap the expressions and factories from the map.
+    val expressions = windowFrameExpressionFactoryPairs.flatMap(_._1)
+    val factories = windowFrameExpressionFactoryPairs.map(_._2).toArray
+
+    // Start processing.
+    child.execute().mapPartitions { stream =>
+      new Iterator[InternalRow] {
+
+        // Get all relevant projections.
+        val result = createResultProjection(expressions)
+        val grouping = UnsafeProjection.create(partitionSpec, child.output)
+
+        // Manage the stream and the grouping.
+        var nextRow: UnsafeRow = null
+        var nextGroup: UnsafeRow = null
+        var nextRowAvailable: Boolean = false
+        private[this] def fetchNextRow() {
+          nextRowAvailable = stream.hasNext
+          if (nextRowAvailable) {
+            nextRow = stream.next().asInstanceOf[UnsafeRow]
+            nextGroup = grouping(nextRow)
+          } else {
+            nextRow = null
+            nextGroup = null
+          }
+        }
+        fetchNextRow()
+
+        // Manage the current partition.
+        val rows = ArrayBuffer.empty[UnsafeRow]
+        val inputFields = child.output.length
+        var sorter: UnsafeExternalSorter = null
+        var rowBuffer: RowBuffer = null
+        val windowFunctionResult = new SpecificMutableRow(expressions.map(_.dataType))
+        val frames = factories.map(_(windowFunctionResult))
+        val numFrames = frames.length
+        private[this] def fetchNextPartition() {
+          // Collect all the rows in the current partition.
+          // Before we start to fetch new input rows, make a copy of nextGroup.
+          val currentGroup = nextGroup.copy()
+
+          // clear last partition
+          if (sorter != null) {
+            // the last sorter of this task will be cleaned up via task completion listener
+            sorter.cleanupResources()
+            sorter = null
+          } else {
+            rows.clear()
+          }
+
+          while (nextRowAvailable && nextGroup == currentGroup) {
+            if (sorter == null) {
+              rows += nextRow.copy()
+
+              if (rows.length >= 4096) {
+                // We will not sort the rows, so prefixComparator and recordComparator are null.
+                sorter = UnsafeExternalSorter.create(
+                  TaskContext.get().taskMemoryManager(),
+                  SparkEnv.get.blockManager,
+                  SparkEnv.get.serializerManager,
+                  TaskContext.get(),
+                  null,
+                  null,
+                  1024,
+                  SparkEnv.get.memoryManager.pageSizeBytes,
+                  false)
+                rows.foreach { r =>
+                  sorter.insertRecord(r.getBaseObject, r.getBaseOffset, r.getSizeInBytes, 0)
+                }
+                rows.clear()
+              }
+            } else {
+              sorter.insertRecord(nextRow.getBaseObject, nextRow.getBaseOffset,
+                nextRow.getSizeInBytes, 0)
+            }
+            fetchNextRow()
+          }
+          if (sorter != null) {
+            rowBuffer = new ExternalRowBuffer(sorter, inputFields)
+          } else {
+            rowBuffer = new ArrayRowBuffer(rows)
+          }
+
+          // Setup the frames.
+          var i = 0
+          while (i < numFrames) {
+            frames(i).prepare(rowBuffer.copy())
+            i += 1
+          }
+
+          // Setup iteration
+          rowIndex = 0
+          rowsSize = rowBuffer.size()
+        }
+
+        // Iteration
+        var rowIndex = 0
+        var rowsSize = 0L
+
+        override final def hasNext: Boolean = rowIndex < rowsSize || nextRowAvailable
+
+        val join = new JoinedRow
+        override final def next(): InternalRow = {
+          // Load the next partition if we need to.
+          if (rowIndex >= rowsSize && nextRowAvailable) {
+            fetchNextPartition()
+          }
+
+          if (rowIndex < rowsSize) {
+            // Get the results for the window frames.
+            var i = 0
+            val current = rowBuffer.next()
+            while (i < numFrames) {
+              frames(i).write(rowIndex, current)
+              i += 1
+            }
+
+            // 'Merge' the input row with the window function result
+            join(current, windowFunctionResult)
+            rowIndex += 1
+
+            // Return the projection.
+            result(join)
+          } else throw new NoSuchElementException
+        }
+      }
+    }
+  }
+}
+
+/**
+ * Function for comparing boundary values.
+ */
+private[execution] abstract class BoundOrdering {
+  def compare(inputRow: InternalRow, inputIndex: Int, outputRow: InternalRow, outputIndex: Int): Int
+}
+
+/**
+ * Compare the input index to the bound of the output index.
+ */
+private[execution] final case class RowBoundOrdering(offset: Int) extends BoundOrdering {
+  override def compare(
+      inputRow: InternalRow,
+      inputIndex: Int,
+      outputRow: InternalRow,
+      outputIndex: Int): Int =
+    inputIndex - (outputIndex + offset)
+}
+
+/**
+ * Compare the value of the input index to the value bound of the output index.
+ */
+private[execution] final case class RangeBoundOrdering(
+    ordering: Ordering[InternalRow],
+    current: Projection,
+    bound: Projection) extends BoundOrdering {
+  override def compare(
+      inputRow: InternalRow,
+      inputIndex: Int,
+      outputRow: InternalRow,
+      outputIndex: Int): Int =
+    ordering.compare(current(inputRow), bound(outputRow))
+}
+
+/**
+ * The interface of row buffer for a partition
+ */
+private[execution] abstract class RowBuffer {
+
+  /** Number of rows. */
+  def size(): Int
+
+  /** Return next row in the buffer, null if no more left. */
+  def next(): InternalRow
+
+  /** Skip the next `n` rows. */
+  def skip(n: Int): Unit
+
+  /** Return a new RowBuffer that has the same rows. */
+  def copy(): RowBuffer
+}
+
+/**
+ * A row buffer based on ArrayBuffer (the number of rows is limited)
+ */
+private[execution] class ArrayRowBuffer(buffer: ArrayBuffer[UnsafeRow]) extends RowBuffer {
+
+  private[this] var cursor: Int = -1
+
+  /** Number of rows. */
+  def size(): Int = buffer.length
+
+  /** Return next row in the buffer, null if no more left. */
+  def next(): InternalRow = {
+    cursor += 1
+    if (cursor < buffer.length) {
+      buffer(cursor)
+    } else {
+      null
+    }
+  }
+
+  /** Skip the next `n` rows. */
+  def skip(n: Int): Unit = {
+    cursor += n
+  }
+
+  /** Return a new RowBuffer that has the same rows. */
+  def copy(): RowBuffer = {
+    new ArrayRowBuffer(buffer)
+  }
+}
+
+/**
+ * An external buffer of rows based on UnsafeExternalSorter
+ */
+private[execution] class ExternalRowBuffer(sorter: UnsafeExternalSorter, numFields: Int)
+  extends RowBuffer {
+
+  private[this] val iter: UnsafeSorterIterator = sorter.getIterator
+
+  private[this] val currentRow = new UnsafeRow(numFields)
+
+  /** Number of rows. */
+  def size(): Int = iter.getNumRecords()
+
+  /** Return next row in the buffer, null if no more left. */
+  def next(): InternalRow = {
+    if (iter.hasNext) {
+      iter.loadNext()
+      currentRow.pointTo(iter.getBaseObject, iter.getBaseOffset, iter.getRecordLength)
+      currentRow
+    } else {
+      null
+    }
+  }
+
+  /** Skip the next `n` rows. */
+  def skip(n: Int): Unit = {
+    var i = 0
+    while (i < n && iter.hasNext) {
+      iter.loadNext()
+      i += 1
+    }
+  }
+
+  /** Return a new RowBuffer that has the same rows. */
+  def copy(): RowBuffer = {
+    new ExternalRowBuffer(sorter, numFields)
+  }
+}
+
+/**
+ * A window function calculates the results of a number of window functions for a window frame.
+ * Before use a frame must be prepared by passing it all the rows in the current partition. After
+ * preparation the update method can be called to fill the output rows.
+ */
+private[execution] abstract class WindowFunctionFrame {
+  /**
+   * Prepare the frame for calculating the results for a partition.
+   *
+   * @param rows to calculate the frame results for.
+   */
+  def prepare(rows: RowBuffer): Unit
+
+  /**
+   * Write the current results to the target row.
+   */
+  def write(index: Int, current: InternalRow): Unit
+}
+
+/**
+ * The offset window frame calculates frames containing LEAD/LAG statements.
+ *
+ * @param target to write results to.
+ * @param expressions to shift a number of rows.
+ * @param inputSchema required for creating a projection.
+ * @param newMutableProjection function used to create the projection.
+ * @param offset by which rows get moved within a partition.
+ */
+private[execution] final class OffsetWindowFunctionFrame(
+    target: MutableRow,
+    ordinal: Int,
+    expressions: Array[Expression],
+    inputSchema: Seq[Attribute],
+    newMutableProjection: (Seq[Expression], Seq[Attribute]) => MutableProjection,
+    offset: Int) extends WindowFunctionFrame {
+
+  /** Rows of the partition currently being processed. */
+  private[this] var input: RowBuffer = null
+
+  /** Index of the input row currently used for output. */
+  private[this] var inputIndex = 0
+
+  /** Row used when there is no valid input. */
+  private[this] val emptyRow = new GenericInternalRow(inputSchema.size)
+
+  /** Row used to combine the offset and the current row. */
+  private[this] val join = new JoinedRow
+
+  /** Create the projection. */
+  private[this] val projection = {
+    // Collect the expressions and bind them.
+    val inputAttrs = inputSchema.map(_.withNullability(true))
+    val numInputAttributes = inputAttrs.size
+    val boundExpressions = Seq.fill(ordinal)(NoOp) ++ expressions.toSeq.map {
+      case e: OffsetWindowFunction =>
+        val input = BindReferences.bindReference(e.input, inputAttrs)
+        if (e.default == null || e.default.foldable && e.default.eval() == null) {
+          // Without default value.
+          input
+        } else {
+          // With default value.
+          val default = BindReferences.bindReference(e.default, inputAttrs).transform {
+            // Shift the input reference to its default version.
+            case BoundReference(o, dataType, nullable) =>
+              BoundReference(o + numInputAttributes, dataType, nullable)
+          }
+          org.apache.spark.sql.catalyst.expressions.Coalesce(input :: default :: Nil)
+        }
+      case e =>
+        BindReferences.bindReference(e, inputAttrs)
+    }
+
+    // Create the projection.
+    newMutableProjection(boundExpressions, Nil).target(target)
+  }
+
+  override def prepare(rows: RowBuffer): Unit = {
+    input = rows
+    // drain the first few rows if offset is larger than zero
+    inputIndex = 0
+    while (inputIndex < offset) {
+      input.next()
+      inputIndex += 1
+    }
+    inputIndex = offset
+  }
+
+  override def write(index: Int, current: InternalRow): Unit = {
+    if (inputIndex >= 0 && inputIndex < input.size) {
+      val r = input.next()
+      join(r, current)
+    } else {
+      join(emptyRow, current)
+    }
+    projection(join)
+    inputIndex += 1
+  }
+}
+
+/**
+ * The sliding window frame calculates frames with the following SQL form:
+ * ... BETWEEN 1 PRECEDING AND 1 FOLLOWING
+ *
+ * @param target to write results to.
+ * @param processor to calculate the row values with.
+ * @param lbound comparator used to identify the lower bound of an output row.
+ * @param ubound comparator used to identify the upper bound of an output row.
+ */
+private[execution] final class SlidingWindowFunctionFrame(
+    target: MutableRow,
+    processor: AggregateProcessor,
+    lbound: BoundOrdering,
+    ubound: BoundOrdering) extends WindowFunctionFrame {
+
+  /** Rows of the partition currently being processed. */
+  private[this] var input: RowBuffer = null
+
+  /** The next row from `input`. */
+  private[this] var nextRow: InternalRow = null
+
+  /** The rows within current sliding window. */
+  private[this] val buffer = new util.ArrayDeque[InternalRow]()
+
+  /**
+   * Index of the first input row with a value greater than the upper bound of the current
+   * output row.
+   */
+  private[this] var inputHighIndex = 0
+
+  /**
+   * Index of the first input row with a value equal to or greater than the lower bound of the
+   * current output row.
+   */
+  private[this] var inputLowIndex = 0
+
+  /** Prepare the frame for calculating a new partition. Reset all variables. */
+  override def prepare(rows: RowBuffer): Unit = {
+    input = rows
+    nextRow = rows.next()
+    inputHighIndex = 0
+    inputLowIndex = 0
+    buffer.clear()
+  }
+
+  /** Write the frame columns for the current row to the given target row. */
+  override def write(index: Int, current: InternalRow): Unit = {
+    var bufferUpdated = index == 0
+
+    // Add all rows to the buffer for which the input row value is equal to or less than
+    // the output row upper bound.
+    while (nextRow != null && ubound.compare(nextRow, inputHighIndex, current, index) <= 0) {
+      buffer.add(nextRow.copy())
+      nextRow = input.next()
+      inputHighIndex += 1
+      bufferUpdated = true
+    }
+
+    // Drop all rows from the buffer for which the input row value is smaller than
+    // the output row lower bound.
+    while (!buffer.isEmpty && lbound.compare(buffer.peek(), inputLowIndex, current, index) < 0) {
+      buffer.remove()
+      inputLowIndex += 1
+      bufferUpdated = true
+    }
+
+    // Only recalculate and update when the buffer changes.
+    if (bufferUpdated) {
+      processor.initialize(input.size)
+      val iter = buffer.iterator()
+      while (iter.hasNext) {
+        processor.update(iter.next())
+      }
+      processor.evaluate(target)
+    }
+  }
+}
+
+/**
+ * The unbounded window frame calculates frames with the following SQL forms:
+ * ... (No Frame Definition)
+ * ... BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
+ *
+ * Its results are  the same for each and every row in the partition. This class can be seen as a
+ * special case of a sliding window, but is optimized for the unbound case.
+ *
+ * @param target to write results to.
+ * @param processor to calculate the row values with.
+ */
+private[execution] final class UnboundedWindowFunctionFrame(
+    target: MutableRow,
+    processor: AggregateProcessor) extends WindowFunctionFrame {
+
+  /** Prepare the frame for calculating a new partition. Process all rows eagerly. */
+  override def prepare(rows: RowBuffer): Unit = {
+    val size = rows.size()
+    processor.initialize(size)
+    var i = 0
+    while (i < size) {
+      processor.update(rows.next())
+      i += 1
+    }
+  }
+
+  /** Write the frame columns for the current row to the given target row. */
+  override def write(index: Int, current: InternalRow): Unit = {
+    // Unfortunately we cannot assume that evaluation is deterministic. So we need to re-evaluate
+    // for each row.
+    processor.evaluate(target)
+  }
+}
+
+/**
+ * The UnboundPreceding window frame calculates frames with the following SQL form:
+ * ... BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+ *
+ * There is only an upper bound. Very common use cases are for instance running sums or counts
+ * (row_number). Technically this is a special case of a sliding window. However a sliding window
+ * has to maintain a buffer, and it must do a full evaluation everytime the buffer changes. This
+ * is not the case when there is no lower bound, given the additive nature of most aggregates
+ * streaming updates and partial evaluation suffice and no buffering is needed.
+ *
+ * @param target to write results to.
+ * @param processor to calculate the row values with.
+ * @param ubound comparator used to identify the upper bound of an output row.
+ */
+private[execution] final class UnboundedPrecedingWindowFunctionFrame(
+    target: MutableRow,
+    processor: AggregateProcessor,
+    ubound: BoundOrdering) extends WindowFunctionFrame {
+
+  /** Rows of the partition currently being processed. */
+  private[this] var input: RowBuffer = null
+
+  /** The next row from `input`. */
+  private[this] var nextRow: InternalRow = null
+
+  /**
+   * Index of the first input row with a value greater than the upper bound of the current
+   * output row.
+   */
+  private[this] var inputIndex = 0
+
+  /** Prepare the frame for calculating a new partition. */
+  override def prepare(rows: RowBuffer): Unit = {
+    input = rows
+    nextRow = rows.next()
+    inputIndex = 0
+    processor.initialize(input.size)
+  }
+
+  /** Write the frame columns for the current row to the given target row. */
+  override def write(index: Int, current: InternalRow): Unit = {
+    var bufferUpdated = index == 0
+
+    // Add all rows to the aggregates for which the input row value is equal to or less than
+    // the output row upper bound.
+    while (nextRow != null && ubound.compare(nextRow, inputIndex, current, index) <= 0) {
+      processor.update(nextRow)
+      nextRow = input.next()
+      inputIndex += 1
+      bufferUpdated = true
+    }
+
+    // Only recalculate and update when the buffer changes.
+    if (bufferUpdated) {
+      processor.evaluate(target)
+    }
+  }
+}
+
+/**
+ * The UnboundFollowing window frame calculates frames with the following SQL form:
+ * ... BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING
+ *
+ * There is only an upper bound. This is a slightly modified version of the sliding window. The
+ * sliding window operator has to check if both upper and the lower bound change when a new row
+ * gets processed, where as the unbounded following only has to check the lower bound.
+ *
+ * This is a very expensive operator to use, O(n * (n - 1) /2), because we need to maintain a
+ * buffer and must do full recalculation after each row. Reverse iteration would be possible, if
+ * the commutativity of the used window functions can be guaranteed.
+ *
+ * @param target to write results to.
+ * @param processor to calculate the row values with.
+ * @param lbound comparator used to identify the lower bound of an output row.
+ */
+private[execution] final class UnboundedFollowingWindowFunctionFrame(
+    target: MutableRow,
+    processor: AggregateProcessor,
+    lbound: BoundOrdering) extends WindowFunctionFrame {
+
+  /** Rows of the partition currently being processed. */
+  private[this] var input: RowBuffer = null
+
+  /**
+   * Index of the first input row with a value equal to or greater than the lower bound of the
+   * current output row.
+   */
+  private[this] var inputIndex = 0
+
+  /** Prepare the frame for calculating a new partition. */
+  override def prepare(rows: RowBuffer): Unit = {
+    input = rows
+    inputIndex = 0
+  }
+
+  /** Write the frame columns for the current row to the given target row. */
+  override def write(index: Int, current: InternalRow): Unit = {
+    var bufferUpdated = index == 0
+
+    // Duplicate the input to have a new iterator
+    val tmp = input.copy()
+
+    // Drop all rows from the buffer for which the input row value is smaller than
+    // the output row lower bound.
+    tmp.skip(inputIndex)
+    var nextRow = tmp.next()
+    while (nextRow != null && lbound.compare(nextRow, inputIndex, current, index) < 0) {
+      nextRow = tmp.next()
+      inputIndex += 1
+      bufferUpdated = true
+    }
+
+    // Only recalculate and update when the buffer changes.
+    if (bufferUpdated) {
+      processor.initialize(input.size)
+      while (nextRow != null) {
+        processor.update(nextRow)
+        nextRow = tmp.next()
+      }
+      processor.evaluate(target)
+    }
+  }
+}
+
+/**
+ * This class prepares and manages the processing of a number of [[AggregateFunction]]s within a
+ * single frame. The [[WindowFunctionFrame]] takes care of processing the frame in the correct way,
+ * this reduces the processing of a [[AggregateWindowFunction]] to processing the underlying
+ * [[AggregateFunction]]. All [[AggregateFunction]]s are processed in [[Complete]] mode.
+ *
+ * [[SizeBasedWindowFunction]]s are initialized in a slightly different way. These functions
+ * require the size of the partition processed, this value is exposed to them when the processor is
+ * constructed.
+ *
+ * Processing of distinct aggregates is currently not supported.
+ *
+ * The implementation is split into an object which takes care of construction, and a the actual
+ * processor class.
+ */
+private[execution] object AggregateProcessor {
+  def apply(
+      functions: Array[Expression],
+      ordinal: Int,
+      inputAttributes: Seq[Attribute],
+      newMutableProjection: (Seq[Expression], Seq[Attribute]) => MutableProjection):
+      AggregateProcessor = {
+    val aggBufferAttributes = mutable.Buffer.empty[AttributeReference]
+    val initialValues = mutable.Buffer.empty[Expression]
+    val updateExpressions = mutable.Buffer.empty[Expression]
+    val evaluateExpressions = mutable.Buffer.fill[Expression](ordinal)(NoOp)
+    val imperatives = mutable.Buffer.empty[ImperativeAggregate]
+
+    // SPARK-14244: `SizeBasedWindowFunction`s are firstly created on driver side and then
+    // serialized to executor side. These functions all reference a global singleton window
+    // partition size attribute reference, i.e., `SizeBasedWindowFunction.n`. Here we must collect
+    // the singleton instance created on driver side instead of using executor side
+    // `SizeBasedWindowFunction.n` to avoid binding failure caused by mismatching expression ID.
+    val partitionSize: Option[AttributeReference] = {
+      val aggs = functions.flatMap(_.collectFirst { case f: SizeBasedWindowFunction => f })
+      aggs.headOption.map(_.n)
+    }
+
+    // Check if there are any SizeBasedWindowFunctions. If there are, we add the partition size to
+    // the aggregation buffer. Note that the ordinal of the partition size value will always be 0.
+    partitionSize.foreach { n =>
+      aggBufferAttributes += n
+      initialValues += NoOp
+      updateExpressions += NoOp
+    }
+
+    // Add an AggregateFunction to the AggregateProcessor.
+    functions.foreach {
+      case agg: DeclarativeAggregate =>
+        aggBufferAttributes ++= agg.aggBufferAttributes
+        initialValues ++= agg.initialValues
+        updateExpressions ++= agg.updateExpressions
+        evaluateExpressions += agg.evaluateExpression
+      case agg: ImperativeAggregate =>
+        val offset = aggBufferAttributes.size
+        val imperative = BindReferences.bindReference(agg
+          .withNewInputAggBufferOffset(offset)
+          .withNewMutableAggBufferOffset(offset),
+          inputAttributes)
+        imperatives += imperative
+        aggBufferAttributes ++= imperative.aggBufferAttributes
+        val noOps = Seq.fill(imperative.aggBufferAttributes.size)(NoOp)
+        initialValues ++= noOps
+        updateExpressions ++= noOps
+        evaluateExpressions += imperative
+      case other =>
+        sys.error(s"Unsupported Aggregate Function: $other")
+    }
+
+    // Create the projections.
+    val initialProjection = newMutableProjection(
+      initialValues,
+      partitionSize.toSeq)
+    val updateProjection = newMutableProjection(
+      updateExpressions,
+      aggBufferAttributes ++ inputAttributes)
+    val evaluateProjection = newMutableProjection(
+      evaluateExpressions,
+      aggBufferAttributes)
+
+    // Create the processor
+    new AggregateProcessor(
+      aggBufferAttributes.toArray,
+      initialProjection,
+      updateProjection,
+      evaluateProjection,
+      imperatives.toArray,
+      partitionSize.isDefined)
+  }
+}
+
+/**
+ * This class manages the processing of a number of aggregate functions. See the documentation of
+ * the object for more information.
+ */
+private[execution] final class AggregateProcessor(
+    private[this] val bufferSchema: Array[AttributeReference],
+    private[this] val initialProjection: MutableProjection,
+    private[this] val updateProjection: MutableProjection,
+    private[this] val evaluateProjection: MutableProjection,
+    private[this] val imperatives: Array[ImperativeAggregate],
+    private[this] val trackPartitionSize: Boolean) {
+
+  private[this] val join = new JoinedRow
+  private[this] val numImperatives = imperatives.length
+  private[this] val buffer = new SpecificMutableRow(bufferSchema.toSeq.map(_.dataType))
+  initialProjection.target(buffer)
+  updateProjection.target(buffer)
+
+  /** Create the initial state. */
+  def initialize(size: Int): Unit = {
+    // Some initialization expressions are dependent on the partition size so we have to
+    // initialize the size before initializing all other fields, and we have to pass the buffer to
+    // the initialization projection.
+    if (trackPartitionSize) {
+      buffer.setInt(0, size)
+    }
+    initialProjection(buffer)
+    var i = 0
+    while (i < numImperatives) {
+      imperatives(i).initialize(buffer)
+      i += 1
+    }
+  }
+
+  /** Update the buffer. */
+  def update(input: InternalRow): Unit = {
+    updateProjection(join(buffer, input))
+    var i = 0
+    while (i < numImperatives) {
+      imperatives(i).update(buffer, input)
+      i += 1
+    }
+  }
+
+  /** Evaluate buffer. */
+  def evaluate(target: MutableRow): Unit =
+    evaluateProjection.target(target)(buffer)
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala
----------------------------------------------------------------------
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/SortBasedAggregate.scala
deleted file mode 100644
index 9fcfea8..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.aggregate
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.errors._
-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.metric.SQLMetrics
-
-case class SortBasedAggregate(
-    requiredChildDistributionExpressions: Option[Seq[Expression]],
-    groupingExpressions: Seq[NamedExpression],
-    aggregateExpressions: Seq[AggregateExpression],
-    aggregateAttributes: Seq[Attribute],
-    initialInputBufferOffset: Int,
-    resultExpressions: Seq[NamedExpression],
-    child: SparkPlan)
-  extends UnaryNode {
-
-  private[this] val aggregateBufferAttributes = {
-    aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)
-  }
-
-  override def producedAttributes: AttributeSet =
-    AttributeSet(aggregateAttributes) ++
-      AttributeSet(resultExpressions.diff(groupingExpressions).map(_.toAttribute)) ++
-      AttributeSet(aggregateBufferAttributes)
-
-  override private[sql] lazy val metrics = Map(
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
-
-  override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute)
-
-  override def requiredChildDistribution: List[Distribution] = {
-    requiredChildDistributionExpressions match {
-      case Some(exprs) if exprs.length == 0 => AllTuples :: Nil
-      case Some(exprs) if exprs.length > 0 => ClusteredDistribution(exprs) :: Nil
-      case None => UnspecifiedDistribution :: Nil
-    }
-  }
-
-  override def requiredChildOrdering: Seq[Seq[SortOrder]] = {
-    groupingExpressions.map(SortOrder(_, Ascending)) :: Nil
-  }
-
-  override def outputOrdering: Seq[SortOrder] = {
-    groupingExpressions.map(SortOrder(_, Ascending))
-  }
-
-  protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") {
-    val numOutputRows = longMetric("numOutputRows")
-    child.execute().mapPartitionsInternal { iter =>
-      // Because the constructor of an aggregation iterator will read at least the first row,
-      // we need to get the value of iter.hasNext first.
-      val hasInput = iter.hasNext
-      if (!hasInput && groupingExpressions.nonEmpty) {
-        // This is a grouped aggregate and the input iterator is empty,
-        // so return an empty iterator.
-        Iterator[UnsafeRow]()
-      } else {
-        val outputIter = new SortBasedAggregationIterator(
-          groupingExpressions,
-          child.output,
-          iter,
-          aggregateExpressions,
-          aggregateAttributes,
-          initialInputBufferOffset,
-          resultExpressions,
-          (expressions, inputSchema) =>
-            newMutableProjection(expressions, inputSchema, subexpressionEliminationEnabled),
-          numOutputRows)
-        if (!hasInput && groupingExpressions.isEmpty) {
-          // There is no input and there is no grouping expressions.
-          // We need to output a single row as the output.
-          numOutputRows += 1
-          Iterator[UnsafeRow](outputIter.outputForEmptyGroupingKeyWithoutInput())
-        } else {
-          outputIter
-        }
-      }
-    }
-  }
-
-  override def simpleString: String = {
-    val allAggregateExpressions = aggregateExpressions
-
-    val keyString = groupingExpressions.mkString("[", ",", "]")
-    val functionString = allAggregateExpressions.mkString("[", ",", "]")
-    val outputString = output.mkString("[", ",", "]")
-    s"SortBasedAggregate(key=$keyString, functions=$functionString, output=$outputString)"
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala
new file mode 100644
index 0000000..3169e0a
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.aggregate
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.errors._
+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, UnaryExecNode}
+import org.apache.spark.sql.execution.metric.SQLMetrics
+
+case class SortBasedAggregateExec(
+    requiredChildDistributionExpressions: Option[Seq[Expression]],
+    groupingExpressions: Seq[NamedExpression],
+    aggregateExpressions: Seq[AggregateExpression],
+    aggregateAttributes: Seq[Attribute],
+    initialInputBufferOffset: Int,
+    resultExpressions: Seq[NamedExpression],
+    child: SparkPlan)
+  extends UnaryExecNode {
+
+  private[this] val aggregateBufferAttributes = {
+    aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)
+  }
+
+  override def producedAttributes: AttributeSet =
+    AttributeSet(aggregateAttributes) ++
+      AttributeSet(resultExpressions.diff(groupingExpressions).map(_.toAttribute)) ++
+      AttributeSet(aggregateBufferAttributes)
+
+  override private[sql] lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
+
+  override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute)
+
+  override def requiredChildDistribution: List[Distribution] = {
+    requiredChildDistributionExpressions match {
+      case Some(exprs) if exprs.length == 0 => AllTuples :: Nil
+      case Some(exprs) if exprs.length > 0 => ClusteredDistribution(exprs) :: Nil
+      case None => UnspecifiedDistribution :: Nil
+    }
+  }
+
+  override def requiredChildOrdering: Seq[Seq[SortOrder]] = {
+    groupingExpressions.map(SortOrder(_, Ascending)) :: Nil
+  }
+
+  override def outputOrdering: Seq[SortOrder] = {
+    groupingExpressions.map(SortOrder(_, Ascending))
+  }
+
+  protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") {
+    val numOutputRows = longMetric("numOutputRows")
+    child.execute().mapPartitionsInternal { iter =>
+      // Because the constructor of an aggregation iterator will read at least the first row,
+      // we need to get the value of iter.hasNext first.
+      val hasInput = iter.hasNext
+      if (!hasInput && groupingExpressions.nonEmpty) {
+        // This is a grouped aggregate and the input iterator is empty,
+        // so return an empty iterator.
+        Iterator[UnsafeRow]()
+      } else {
+        val outputIter = new SortBasedAggregationIterator(
+          groupingExpressions,
+          child.output,
+          iter,
+          aggregateExpressions,
+          aggregateAttributes,
+          initialInputBufferOffset,
+          resultExpressions,
+          (expressions, inputSchema) =>
+            newMutableProjection(expressions, inputSchema, subexpressionEliminationEnabled),
+          numOutputRows)
+        if (!hasInput && groupingExpressions.isEmpty) {
+          // There is no input and there is no grouping expressions.
+          // We need to output a single row as the output.
+          numOutputRows += 1
+          Iterator[UnsafeRow](outputIter.outputForEmptyGroupingKeyWithoutInput())
+        } else {
+          outputIter
+        }
+      }
+    }
+  }
+
+  override def simpleString: String = {
+    val allAggregateExpressions = aggregateExpressions
+
+    val keyString = groupingExpressions.mkString("[", ",", "]")
+    val functionString = allAggregateExpressions.mkString("[", ",", "]")
+    val outputString = output.mkString("[", ",", "]")
+    s"SortBasedAggregate(key=$keyString, functions=$functionString, output=$outputString)"
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala
----------------------------------------------------------------------
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 49b682a..782da0e 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)

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala
----------------------------------------------------------------------
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 4682949..f93c446 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))

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
----------------------------------------------------------------------
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 892c57a..83f527f 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarTableScan.scala
----------------------------------------------------------------------
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/InMemoryColumnarTableScan.scala
deleted file mode 100644
index 1f964b1..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarTableScan.scala
+++ /dev/null
@@ -1,358 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.columnar
-
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.spark.{Accumulable, Accumulator, Accumulators}
-import org.apache.spark.network.util.JavaUtils
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
-import org.apache.spark.sql.catalyst.dsl.expressions._
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.logical
-import org.apache.spark.sql.catalyst.plans.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.metric.SQLMetrics
-import org.apache.spark.sql.types.UserDefinedType
-import org.apache.spark.storage.StorageLevel
-
-private[sql] object InMemoryRelation {
-  def apply(
-      useCompression: Boolean,
-      batchSize: Int,
-      storageLevel: StorageLevel,
-      child: SparkPlan,
-      tableName: Option[String]): InMemoryRelation =
-    new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child, tableName)()
-}
-
-/**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
- */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
-
-private[sql] case class InMemoryRelation(
-    output: Seq[Attribute],
-    useCompression: Boolean,
-    batchSize: Int,
-    storageLevel: StorageLevel,
-    @transient child: SparkPlan,
-    tableName: Option[String])(
-    @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null,
-    @transient private[sql] var _statistics: Statistics = null,
-    private[sql] var _batchStats: Accumulable[ArrayBuffer[InternalRow], InternalRow] = null)
-  extends logical.LeafNode with MultiInstanceRelation {
-
-  override def producedAttributes: AttributeSet = outputSet
-
-  private val batchStats: Accumulable[ArrayBuffer[InternalRow], InternalRow] =
-    if (_batchStats == null) {
-      child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[InternalRow])
-    } else {
-      _batchStats
-    }
-
-  @transient val partitionStatistics = new PartitionStatistics(output)
-
-  private def computeSizeInBytes = {
-    val sizeOfRow: Expression =
-      BindReferences.bindReference(
-        output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add),
-        partitionStatistics.schema)
-
-    batchStats.value.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum
-  }
-
-  // Statistics propagation contracts:
-  // 1. Non-null `_statistics` must reflect the actual statistics of the underlying data
-  // 2. Only propagate statistics when `_statistics` is non-null
-  private def statisticsToBePropagated = if (_statistics == null) {
-    val updatedStats = statistics
-    if (_statistics == null) null else updatedStats
-  } else {
-    _statistics
-  }
-
-  override def statistics: Statistics = {
-    if (_statistics == null) {
-      if (batchStats.value.isEmpty) {
-        // Underlying columnar RDD hasn't been materialized, no useful statistics information
-        // available, return the default statistics.
-        Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes)
-      } else {
-        // Underlying columnar RDD has been materialized, required information has also been
-        // collected via the `batchStats` accumulator, compute the final statistics,
-        // and update `_statistics`.
-        _statistics = Statistics(sizeInBytes = computeSizeInBytes)
-        _statistics
-      }
-    } else {
-      // Pre-computed statistics
-      _statistics
-    }
-  }
-
-  // If the cached column buffers were not passed in, we calculate them in the constructor.
-  // As in Spark, the actual work of caching is lazy.
-  if (_cachedColumnBuffers == null) {
-    buildBuffers()
-  }
-
-  def recache(): Unit = {
-    _cachedColumnBuffers.unpersist()
-    _cachedColumnBuffers = null
-    buildBuffers()
-  }
-
-  private def buildBuffers(): Unit = {
-    val output = child.output
-    val cached = child.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics)
-                        .flatMap(_.values))
-
-          batchStats += stats
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
-    }.persist(storageLevel)
-
-    cached.setName(tableName.map(n => s"In-memory table $n").getOrElse(child.toString))
-    _cachedColumnBuffers = cached
-  }
-
-  def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = {
-    InMemoryRelation(
-      newOutput, useCompression, batchSize, storageLevel, child, tableName)(
-      _cachedColumnBuffers, statisticsToBePropagated, batchStats)
-  }
-
-  override def newInstance(): this.type = {
-    new InMemoryRelation(
-      output.map(_.newInstance()),
-      useCompression,
-      batchSize,
-      storageLevel,
-      child,
-      tableName)(
-      _cachedColumnBuffers,
-      statisticsToBePropagated,
-      batchStats).asInstanceOf[this.type]
-  }
-
-  def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers
-
-  override protected def otherCopyArgs: Seq[AnyRef] =
-    Seq(_cachedColumnBuffers, statisticsToBePropagated, batchStats)
-
-  private[sql] def uncache(blocking: Boolean): Unit = {
-    Accumulators.remove(batchStats.id)
-    cachedColumnBuffers.unpersist(blocking)
-    _cachedColumnBuffers = null
-  }
-}
-
-private[sql] case class InMemoryColumnarTableScan(
-    attributes: Seq[Attribute],
-    predicates: Seq[Expression],
-    @transient relation: InMemoryRelation)
-  extends LeafNode {
-
-  private[sql] override lazy val metrics = Map(
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
-
-  override def output: Seq[Attribute] = attributes
-
-  // The cached version does not change the outputPartitioning of the original SparkPlan.
-  override def outputPartitioning: Partitioning = relation.child.outputPartitioning
-
-  // The cached version does not change the outputOrdering of the original SparkPlan.
-  override def outputOrdering: Seq[SortOrder] = relation.child.outputOrdering
-
-  private def statsFor(a: Attribute) = relation.partitionStatistics.forAttribute(a)
-
-  // Returned filter predicate should return false iff it is impossible for the input expression
-  // to evaluate to `true' based on statistics collected about this partition batch.
-  @transient val buildFilter: PartialFunction[Expression, Expression] = {
-    case And(lhs: Expression, rhs: Expression)
-      if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) =>
-      (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _)
-
-    case Or(lhs: Expression, rhs: Expression)
-      if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) =>
-      buildFilter(lhs) || buildFilter(rhs)
-
-    case EqualTo(a: AttributeReference, l: Literal) =>
-      statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
-    case EqualTo(l: Literal, a: AttributeReference) =>
-      statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
-
-    case LessThan(a: AttributeReference, l: Literal) => statsFor(a).lowerBound < l
-    case LessThan(l: Literal, a: AttributeReference) => l < statsFor(a).upperBound
-
-    case LessThanOrEqual(a: AttributeReference, l: Literal) => statsFor(a).lowerBound <= l
-    case LessThanOrEqual(l: Literal, a: AttributeReference) => l <= statsFor(a).upperBound
-
-    case GreaterThan(a: AttributeReference, l: Literal) => l < statsFor(a).upperBound
-    case GreaterThan(l: Literal, a: AttributeReference) => statsFor(a).lowerBound < l
-
-    case GreaterThanOrEqual(a: AttributeReference, l: Literal) => l <= statsFor(a).upperBound
-    case GreaterThanOrEqual(l: Literal, a: AttributeReference) => statsFor(a).lowerBound <= l
-
-    case IsNull(a: Attribute) => statsFor(a).nullCount > 0
-    case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0
-  }
-
-  val partitionFilters: Seq[Expression] = {
-    predicates.flatMap { p =>
-      val filter = buildFilter.lift(p)
-      val boundFilter =
-        filter.map(
-          BindReferences.bindReference(
-            _,
-            relation.partitionStatistics.schema,
-            allowFailures = true))
-
-      boundFilter.foreach(_ =>
-        filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f")))
-
-      // If the filter can't be resolved then we are missing required statistics.
-      boundFilter.filter(_.resolved)
-    }
-  }
-
-  lazy val enableAccumulators: Boolean =
-    sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean
-
-  // Accumulators used for testing purposes
-  lazy val readPartitions: Accumulator[Int] = sparkContext.accumulator(0)
-  lazy val readBatches: Accumulator[Int] = sparkContext.accumulator(0)
-
-  private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    val numOutputRows = longMetric("numOutputRows")
-
-    if (enableAccumulators) {
-      readPartitions.setValue(0)
-      readBatches.setValue(0)
-    }
-
-    // Using these variables here to avoid serialization of entire objects (if referenced directly)
-    // within the map Partitions closure.
-    val schema = relation.partitionStatistics.schema
-    val schemaIndex = schema.zipWithIndex
-    val relOutput = relation.output
-    val buffers = relation.cachedColumnBuffers
-
-    buffers.mapPartitionsInternal { cachedBatchIterator =>
-      val partitionFilter = newPredicate(
-        partitionFilters.reduceOption(And).getOrElse(Literal(true)),
-        schema)
-
-      // Find the ordinals and data types of the requested columns.
-      val (requestedColumnIndices, requestedColumnDataTypes) =
-        attributes.map { a =>
-          relOutput.indexWhere(_.exprId == a.exprId) -> a.dataType
-        }.unzip
-
-      // Do partition batch pruning if enabled
-      val cachedBatchesToScan =
-        if (inMemoryPartitionPruningEnabled) {
-          cachedBatchIterator.filter { cachedBatch =>
-            if (!partitionFilter(cachedBatch.stats)) {
-              def statsString: String = schemaIndex.map {
-                case (a, i) =>
-                  val value = cachedBatch.stats.get(i, a.dataType)
-                  s"${a.name}: $value"
-              }.mkString(", ")
-              logInfo(s"Skipping partition based on stats $statsString")
-              false
-            } else {
-              if (enableAccumulators) {
-                readBatches += 1
-              }
-              true
-            }
-          }
-        } else {
-          cachedBatchIterator
-        }
-
-      // update SQL metrics
-      val withMetrics = cachedBatchesToScan.map { batch =>
-        numOutputRows += batch.numRows
-        batch
-      }
-
-      val columnTypes = requestedColumnDataTypes.map {
-        case udt: UserDefinedType[_] => udt.sqlType
-        case other => other
-      }.toArray
-      val columnarIterator = GenerateColumnAccessor.generate(columnTypes)
-      columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray)
-      if (enableAccumulators && columnarIterator.hasNext) {
-        readPartitions += 1
-      }
-      columnarIterator
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
new file mode 100644
index 0000000..cb957b9
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.columnar
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{Accumulable, Accumulator, Accumulators}
+import org.apache.spark.network.util.JavaUtils
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.logical.Statistics
+import org.apache.spark.sql.catalyst.plans.physical.Partitioning
+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
+
+private[sql] object InMemoryRelation {
+  def apply(
+      useCompression: Boolean,
+      batchSize: Int,
+      storageLevel: StorageLevel,
+      child: SparkPlan,
+      tableName: Option[String]): InMemoryRelation =
+    new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child, tableName)()
+}
+
+/**
+ * CachedBatch is a cached batch of rows.
+ *
+ * @param numRows The total number of rows in this batch
+ * @param buffers The buffers for serialized columns
+ * @param stats The stat of columns
+ */
+private[columnar]
+case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+
+private[sql] case class InMemoryRelation(
+    output: Seq[Attribute],
+    useCompression: Boolean,
+    batchSize: Int,
+    storageLevel: StorageLevel,
+    @transient child: SparkPlan,
+    tableName: Option[String])(
+    @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null,
+    @transient private[sql] var _statistics: Statistics = null,
+    private[sql] var _batchStats: Accumulable[ArrayBuffer[InternalRow], InternalRow] = null)
+  extends logical.LeafNode with MultiInstanceRelation {
+
+  override def producedAttributes: AttributeSet = outputSet
+
+  private val batchStats: Accumulable[ArrayBuffer[InternalRow], InternalRow] =
+    if (_batchStats == null) {
+      child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[InternalRow])
+    } else {
+      _batchStats
+    }
+
+  @transient val partitionStatistics = new PartitionStatistics(output)
+
+  private def computeSizeInBytes = {
+    val sizeOfRow: Expression =
+      BindReferences.bindReference(
+        output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add),
+        partitionStatistics.schema)
+
+    batchStats.value.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum
+  }
+
+  // Statistics propagation contracts:
+  // 1. Non-null `_statistics` must reflect the actual statistics of the underlying data
+  // 2. Only propagate statistics when `_statistics` is non-null
+  private def statisticsToBePropagated = if (_statistics == null) {
+    val updatedStats = statistics
+    if (_statistics == null) null else updatedStats
+  } else {
+    _statistics
+  }
+
+  override def statistics: Statistics = {
+    if (_statistics == null) {
+      if (batchStats.value.isEmpty) {
+        // Underlying columnar RDD hasn't been materialized, no useful statistics information
+        // available, return the default statistics.
+        Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes)
+      } else {
+        // Underlying columnar RDD has been materialized, required information has also been
+        // collected via the `batchStats` accumulator, compute the final statistics,
+        // and update `_statistics`.
+        _statistics = Statistics(sizeInBytes = computeSizeInBytes)
+        _statistics
+      }
+    } else {
+      // Pre-computed statistics
+      _statistics
+    }
+  }
+
+  // If the cached column buffers were not passed in, we calculate them in the constructor.
+  // As in Spark, the actual work of caching is lazy.
+  if (_cachedColumnBuffers == null) {
+    buildBuffers()
+  }
+
+  def recache(): Unit = {
+    _cachedColumnBuffers.unpersist()
+    _cachedColumnBuffers = null
+    buildBuffers()
+  }
+
+  private def buildBuffers(): Unit = {
+    val output = child.output
+    val cached = child.execute().mapPartitionsInternal { rowIterator =>
+      new Iterator[CachedBatch] {
+        def next(): CachedBatch = {
+          val columnBuilders = output.map { attribute =>
+            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
+          }.toArray
+
+          var rowCount = 0
+          var totalSize = 0L
+          while (rowIterator.hasNext && rowCount < batchSize
+            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
+            val row = rowIterator.next()
+
+            // Added for SPARK-6082. This assertion can be useful for scenarios when something
+            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
+            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
+            // hard to decipher.
+            assert(
+              row.numFields == columnBuilders.length,
+              s"Row column number mismatch, expected ${output.size} columns, " +
+                s"but got ${row.numFields}." +
+                s"\nRow content: $row")
+
+            var i = 0
+            totalSize = 0
+            while (i < row.numFields) {
+              columnBuilders(i).appendFrom(row, i)
+              totalSize += columnBuilders(i).columnStats.sizeInBytes
+              i += 1
+            }
+            rowCount += 1
+          }
+
+          val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics)
+                        .flatMap(_.values))
+
+          batchStats += stats
+          CachedBatch(rowCount, columnBuilders.map { builder =>
+            JavaUtils.bufferToArray(builder.build())
+          }, stats)
+        }
+
+        def hasNext: Boolean = rowIterator.hasNext
+      }
+    }.persist(storageLevel)
+
+    cached.setName(tableName.map(n => s"In-memory table $n").getOrElse(child.toString))
+    _cachedColumnBuffers = cached
+  }
+
+  def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = {
+    InMemoryRelation(
+      newOutput, useCompression, batchSize, storageLevel, child, tableName)(
+      _cachedColumnBuffers, statisticsToBePropagated, batchStats)
+  }
+
+  override def newInstance(): this.type = {
+    new InMemoryRelation(
+      output.map(_.newInstance()),
+      useCompression,
+      batchSize,
+      storageLevel,
+      child,
+      tableName)(
+      _cachedColumnBuffers,
+      statisticsToBePropagated,
+      batchStats).asInstanceOf[this.type]
+  }
+
+  def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers
+
+  override protected def otherCopyArgs: Seq[AnyRef] =
+    Seq(_cachedColumnBuffers, statisticsToBePropagated, batchStats)
+
+  private[sql] def uncache(blocking: Boolean): Unit = {
+    Accumulators.remove(batchStats.id)
+    cachedColumnBuffers.unpersist(blocking)
+    _cachedColumnBuffers = null
+  }
+}
+
+private[sql] case class InMemoryTableScanExec(
+    attributes: Seq[Attribute],
+    predicates: Seq[Expression],
+    @transient relation: InMemoryRelation)
+  extends LeafExecNode {
+
+  private[sql] override lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
+
+  override def output: Seq[Attribute] = attributes
+
+  // The cached version does not change the outputPartitioning of the original SparkPlan.
+  override def outputPartitioning: Partitioning = relation.child.outputPartitioning
+
+  // The cached version does not change the outputOrdering of the original SparkPlan.
+  override def outputOrdering: Seq[SortOrder] = relation.child.outputOrdering
+
+  private def statsFor(a: Attribute) = relation.partitionStatistics.forAttribute(a)
+
+  // Returned filter predicate should return false iff it is impossible for the input expression
+  // to evaluate to `true' based on statistics collected about this partition batch.
+  @transient val buildFilter: PartialFunction[Expression, Expression] = {
+    case And(lhs: Expression, rhs: Expression)
+      if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) =>
+      (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _)
+
+    case Or(lhs: Expression, rhs: Expression)
+      if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) =>
+      buildFilter(lhs) || buildFilter(rhs)
+
+    case EqualTo(a: AttributeReference, l: Literal) =>
+      statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+    case EqualTo(l: Literal, a: AttributeReference) =>
+      statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+
+    case LessThan(a: AttributeReference, l: Literal) => statsFor(a).lowerBound < l
+    case LessThan(l: Literal, a: AttributeReference) => l < statsFor(a).upperBound
+
+    case LessThanOrEqual(a: AttributeReference, l: Literal) => statsFor(a).lowerBound <= l
+    case LessThanOrEqual(l: Literal, a: AttributeReference) => l <= statsFor(a).upperBound
+
+    case GreaterThan(a: AttributeReference, l: Literal) => l < statsFor(a).upperBound
+    case GreaterThan(l: Literal, a: AttributeReference) => statsFor(a).lowerBound < l
+
+    case GreaterThanOrEqual(a: AttributeReference, l: Literal) => l <= statsFor(a).upperBound
+    case GreaterThanOrEqual(l: Literal, a: AttributeReference) => statsFor(a).lowerBound <= l
+
+    case IsNull(a: Attribute) => statsFor(a).nullCount > 0
+    case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0
+  }
+
+  val partitionFilters: Seq[Expression] = {
+    predicates.flatMap { p =>
+      val filter = buildFilter.lift(p)
+      val boundFilter =
+        filter.map(
+          BindReferences.bindReference(
+            _,
+            relation.partitionStatistics.schema,
+            allowFailures = true))
+
+      boundFilter.foreach(_ =>
+        filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f")))
+
+      // If the filter can't be resolved then we are missing required statistics.
+      boundFilter.filter(_.resolved)
+    }
+  }
+
+  lazy val enableAccumulators: Boolean =
+    sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean
+
+  // Accumulators used for testing purposes
+  lazy val readPartitions: Accumulator[Int] = sparkContext.accumulator(0)
+  lazy val readBatches: Accumulator[Int] = sparkContext.accumulator(0)
+
+  private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    val numOutputRows = longMetric("numOutputRows")
+
+    if (enableAccumulators) {
+      readPartitions.setValue(0)
+      readBatches.setValue(0)
+    }
+
+    // Using these variables here to avoid serialization of entire objects (if referenced directly)
+    // within the map Partitions closure.
+    val schema = relation.partitionStatistics.schema
+    val schemaIndex = schema.zipWithIndex
+    val relOutput = relation.output
+    val buffers = relation.cachedColumnBuffers
+
+    buffers.mapPartitionsInternal { cachedBatchIterator =>
+      val partitionFilter = newPredicate(
+        partitionFilters.reduceOption(And).getOrElse(Literal(true)),
+        schema)
+
+      // Find the ordinals and data types of the requested columns.
+      val (requestedColumnIndices, requestedColumnDataTypes) =
+        attributes.map { a =>
+          relOutput.indexWhere(_.exprId == a.exprId) -> a.dataType
+        }.unzip
+
+      // Do partition batch pruning if enabled
+      val cachedBatchesToScan =
+        if (inMemoryPartitionPruningEnabled) {
+          cachedBatchIterator.filter { cachedBatch =>
+            if (!partitionFilter(cachedBatch.stats)) {
+              def statsString: String = schemaIndex.map {
+                case (a, i) =>
+                  val value = cachedBatch.stats.get(i, a.dataType)
+                  s"${a.name}: $value"
+              }.mkString(", ")
+              logInfo(s"Skipping partition based on stats $statsString")
+              false
+            } else {
+              if (enableAccumulators) {
+                readBatches += 1
+              }
+              true
+            }
+          }
+        } else {
+          cachedBatchIterator
+        }
+
+      // update SQL metrics
+      val withMetrics = cachedBatchesToScan.map { batch =>
+        numOutputRows += batch.numRows
+        batch
+      }
+
+      val columnTypes = requestedColumnDataTypes.map {
+        case udt: UserDefinedType[_] => udt.sqlType
+        case other => other
+      }.toArray
+      val columnarIterator = GenerateColumnAccessor.generate(columnTypes)
+      columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray)
+      if (enableAccumulators && columnarIterator.hasNext) {
+        readPartitions += 1
+      }
+      columnarIterator
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[3/7] spark git commit: [SPARK-14855][SQL] Add "Exec" suffix to physical operators

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala
----------------------------------------------------------------------
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/ShuffledHashJoin.scala
deleted file mode 100644
index 785373b..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.joins
-
-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.{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.metric.SQLMetrics
-
-/**
- * Performs a hash join of two child relations by first shuffling the data using the join keys.
- */
-case class ShuffledHashJoin(
-    leftKeys: Seq[Expression],
-    rightKeys: Seq[Expression],
-    joinType: JoinType,
-    buildSide: BuildSide,
-    condition: Option[Expression],
-    left: SparkPlan,
-    right: SparkPlan)
-  extends BinaryNode with HashJoin {
-
-  override private[sql] lazy val metrics = Map(
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"),
-    "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size of build side"),
-    "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map"))
-
-  override def outputPartitioning: Partitioning = joinType match {
-    case Inner => PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning))
-    case LeftAnti => left.outputPartitioning
-    case LeftSemi => left.outputPartitioning
-    case LeftOuter => left.outputPartitioning
-    case RightOuter => right.outputPartitioning
-    case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions)
-    case x =>
-      throw new IllegalArgumentException(s"ShuffledHashJoin should not take $x as the JoinType")
-  }
-
-  override def requiredChildDistribution: Seq[Distribution] =
-    ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil
-
-  private def buildHashedRelation(iter: Iterator[InternalRow]): HashedRelation = {
-    val buildDataSize = longMetric("buildDataSize")
-    val buildTime = longMetric("buildTime")
-    val start = System.nanoTime()
-    val context = TaskContext.get()
-    val relation = HashedRelation(iter, buildKeys, taskMemoryManager = context.taskMemoryManager())
-    buildTime += (System.nanoTime() - start) / 1000000
-    buildDataSize += relation.estimatedSize
-    // This relation is usually used until the end of task.
-    context.addTaskCompletionListener(_ => relation.close())
-    relation
-  }
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    val numOutputRows = longMetric("numOutputRows")
-    streamedPlan.execute().zipPartitions(buildPlan.execute()) { (streamIter, buildIter) =>
-      val hashed = buildHashedRelation(buildIter)
-      join(streamIter, hashed, numOutputRows)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala
new file mode 100644
index 0000000..68cd3cb
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.joins
+
+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.{BindReferences, Expression, UnsafeRow}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.physical._
+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 ShuffledHashJoinExec(
+    leftKeys: Seq[Expression],
+    rightKeys: Seq[Expression],
+    joinType: JoinType,
+    buildSide: BuildSide,
+    condition: Option[Expression],
+    left: SparkPlan,
+    right: SparkPlan)
+  extends BinaryExecNode with HashJoin {
+
+  override private[sql] lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"),
+    "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size of build side"),
+    "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map"))
+
+  override def outputPartitioning: Partitioning = joinType match {
+    case Inner => PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning))
+    case LeftAnti => left.outputPartitioning
+    case LeftSemi => left.outputPartitioning
+    case LeftOuter => left.outputPartitioning
+    case RightOuter => right.outputPartitioning
+    case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions)
+    case x =>
+      throw new IllegalArgumentException(s"ShuffledHashJoin should not take $x as the JoinType")
+  }
+
+  override def requiredChildDistribution: Seq[Distribution] =
+    ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil
+
+  private def buildHashedRelation(iter: Iterator[InternalRow]): HashedRelation = {
+    val buildDataSize = longMetric("buildDataSize")
+    val buildTime = longMetric("buildTime")
+    val start = System.nanoTime()
+    val context = TaskContext.get()
+    val relation = HashedRelation(iter, buildKeys, taskMemoryManager = context.taskMemoryManager())
+    buildTime += (System.nanoTime() - start) / 1000000
+    buildDataSize += relation.estimatedSize
+    // This relation is usually used until the end of task.
+    context.addTaskCompletionListener(_ => relation.close())
+    relation
+  }
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    val numOutputRows = longMetric("numOutputRows")
+    streamedPlan.execute().zipPartitions(buildPlan.execute()) { (streamIter, buildIter) =>
+      val hashed = buildHashedRelation(buildIter)
+      join(streamIter, hashed, numOutputRows)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala
----------------------------------------------------------------------
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/SortMergeJoin.scala
deleted file mode 100644
index 4e45fd6..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala
+++ /dev/null
@@ -1,964 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.joins
-
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.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.metric.{LongSQLMetric, SQLMetrics}
-import org.apache.spark.util.collection.BitSet
-
-/**
- * Performs an sort merge join of two child relations.
- */
-case class SortMergeJoin(
-    leftKeys: Seq[Expression],
-    rightKeys: Seq[Expression],
-    joinType: JoinType,
-    condition: Option[Expression],
-    left: SparkPlan,
-    right: SparkPlan) extends BinaryNode with CodegenSupport {
-
-  override private[sql] lazy val metrics = Map(
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
-
-  override def output: Seq[Attribute] = {
-    joinType match {
-      case Inner =>
-        left.output ++ right.output
-      case LeftOuter =>
-        left.output ++ right.output.map(_.withNullability(true))
-      case RightOuter =>
-        left.output.map(_.withNullability(true)) ++ right.output
-      case FullOuter =>
-        (left.output ++ right.output).map(_.withNullability(true))
-      case x =>
-        throw new IllegalArgumentException(
-          s"${getClass.getSimpleName} should not take $x as the JoinType")
-    }
-  }
-
-  override def outputPartitioning: Partitioning = joinType match {
-    case Inner => PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning))
-    // For left and right outer joins, the output is partitioned by the streamed input's join keys.
-    case LeftOuter => left.outputPartitioning
-    case RightOuter => right.outputPartitioning
-    case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions)
-    case x =>
-      throw new IllegalArgumentException(
-        s"${getClass.getSimpleName} should not take $x as the JoinType")
-  }
-
-  override def requiredChildDistribution: Seq[Distribution] =
-    ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil
-
-  override def outputOrdering: Seq[SortOrder] = requiredOrders(leftKeys)
-
-  override def requiredChildOrdering: Seq[Seq[SortOrder]] =
-    requiredOrders(leftKeys) :: requiredOrders(rightKeys) :: Nil
-
-  private def requiredOrders(keys: Seq[Expression]): Seq[SortOrder] = {
-    // This must be ascending in order to agree with the `keyOrdering` defined in `doExecute()`.
-    keys.map(SortOrder(_, Ascending))
-  }
-
-  private def createLeftKeyGenerator(): Projection =
-    UnsafeProjection.create(leftKeys, left.output)
-
-  private def createRightKeyGenerator(): Projection =
-    UnsafeProjection.create(rightKeys, right.output)
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    val numOutputRows = longMetric("numOutputRows")
-
-    left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) =>
-      val boundCondition: (InternalRow) => Boolean = {
-        condition.map { cond =>
-          newPredicate(cond, left.output ++ right.output)
-        }.getOrElse {
-          (r: InternalRow) => true
-        }
-      }
-      // An ordering that can be used to compare keys from both sides.
-      val keyOrdering = newNaturalAscendingOrdering(leftKeys.map(_.dataType))
-      val resultProj: InternalRow => InternalRow = UnsafeProjection.create(output, output)
-
-      joinType match {
-        case Inner =>
-          new RowIterator {
-            // The projection used to extract keys from input rows of the left child.
-            private[this] val leftKeyGenerator = UnsafeProjection.create(leftKeys, left.output)
-
-            // The projection used to extract keys from input rows of the right child.
-            private[this] val rightKeyGenerator = UnsafeProjection.create(rightKeys, right.output)
-
-            // An ordering that can be used to compare keys from both sides.
-            private[this] val keyOrdering = newNaturalAscendingOrdering(leftKeys.map(_.dataType))
-            private[this] var currentLeftRow: InternalRow = _
-            private[this] var currentRightMatches: ArrayBuffer[InternalRow] = _
-            private[this] var currentMatchIdx: Int = -1
-            private[this] val smjScanner = new SortMergeJoinScanner(
-              leftKeyGenerator,
-              rightKeyGenerator,
-              keyOrdering,
-              RowIterator.fromScala(leftIter),
-              RowIterator.fromScala(rightIter)
-            )
-            private[this] val joinRow = new JoinedRow
-            private[this] val resultProjection: (InternalRow) => InternalRow =
-              UnsafeProjection.create(schema)
-
-            if (smjScanner.findNextInnerJoinRows()) {
-              currentRightMatches = smjScanner.getBufferedMatches
-              currentLeftRow = smjScanner.getStreamedRow
-              currentMatchIdx = 0
-            }
-
-            override def advanceNext(): Boolean = {
-              while (currentMatchIdx >= 0) {
-                if (currentMatchIdx == currentRightMatches.length) {
-                  if (smjScanner.findNextInnerJoinRows()) {
-                    currentRightMatches = smjScanner.getBufferedMatches
-                    currentLeftRow = smjScanner.getStreamedRow
-                    currentMatchIdx = 0
-                  } else {
-                    currentRightMatches = null
-                    currentLeftRow = null
-                    currentMatchIdx = -1
-                    return false
-                  }
-                }
-                joinRow(currentLeftRow, currentRightMatches(currentMatchIdx))
-                currentMatchIdx += 1
-                if (boundCondition(joinRow)) {
-                  numOutputRows += 1
-                  return true
-                }
-              }
-              false
-            }
-
-            override def getRow: InternalRow = resultProjection(joinRow)
-          }.toScala
-
-        case LeftOuter =>
-          val smjScanner = new SortMergeJoinScanner(
-            streamedKeyGenerator = createLeftKeyGenerator(),
-            bufferedKeyGenerator = createRightKeyGenerator(),
-            keyOrdering,
-            streamedIter = RowIterator.fromScala(leftIter),
-            bufferedIter = RowIterator.fromScala(rightIter)
-          )
-          val rightNullRow = new GenericInternalRow(right.output.length)
-          new LeftOuterIterator(
-            smjScanner, rightNullRow, boundCondition, resultProj, numOutputRows).toScala
-
-        case RightOuter =>
-          val smjScanner = new SortMergeJoinScanner(
-            streamedKeyGenerator = createRightKeyGenerator(),
-            bufferedKeyGenerator = createLeftKeyGenerator(),
-            keyOrdering,
-            streamedIter = RowIterator.fromScala(rightIter),
-            bufferedIter = RowIterator.fromScala(leftIter)
-          )
-          val leftNullRow = new GenericInternalRow(left.output.length)
-          new RightOuterIterator(
-            smjScanner, leftNullRow, boundCondition, resultProj, numOutputRows).toScala
-
-        case FullOuter =>
-          val leftNullRow = new GenericInternalRow(left.output.length)
-          val rightNullRow = new GenericInternalRow(right.output.length)
-          val smjScanner = new SortMergeFullOuterJoinScanner(
-            leftKeyGenerator = createLeftKeyGenerator(),
-            rightKeyGenerator = createRightKeyGenerator(),
-            keyOrdering,
-            leftIter = RowIterator.fromScala(leftIter),
-            rightIter = RowIterator.fromScala(rightIter),
-            boundCondition,
-            leftNullRow,
-            rightNullRow)
-
-          new FullOuterIterator(
-            smjScanner,
-            resultProj,
-            numOutputRows).toScala
-
-        case x =>
-          throw new IllegalArgumentException(
-            s"SortMergeJoin should not take $x as the JoinType")
-      }
-
-    }
-  }
-
-  override def supportCodegen: Boolean = {
-    joinType == Inner
-  }
-
-  override def inputRDDs(): Seq[RDD[InternalRow]] = {
-    left.execute() :: right.execute() :: Nil
-  }
-
-  private def createJoinKey(
-      ctx: CodegenContext,
-      row: String,
-      keys: Seq[Expression],
-      input: Seq[Attribute]): Seq[ExprCode] = {
-    ctx.INPUT_ROW = row
-    keys.map(BindReferences.bindReference(_, input).genCode(ctx))
-  }
-
-  private def copyKeys(ctx: CodegenContext, vars: Seq[ExprCode]): Seq[ExprCode] = {
-    vars.zipWithIndex.map { case (ev, i) =>
-      val value = ctx.freshName("value")
-      ctx.addMutableState(ctx.javaType(leftKeys(i).dataType), value, "")
-      val code =
-        s"""
-           |$value = ${ev.value};
-         """.stripMargin
-      ExprCode(code, "false", value)
-    }
-  }
-
-  private def genComparision(ctx: CodegenContext, a: Seq[ExprCode], b: Seq[ExprCode]): String = {
-    val comparisons = a.zip(b).zipWithIndex.map { case ((l, r), i) =>
-      s"""
-         |if (comp == 0) {
-         |  comp = ${ctx.genComp(leftKeys(i).dataType, l.value, r.value)};
-         |}
-       """.stripMargin.trim
-    }
-    s"""
-       |comp = 0;
-       |${comparisons.mkString("\n")}
-     """.stripMargin
-  }
-
-  /**
-   * Generate a function to scan both left and right to find a match, returns the term for
-   * matched one row from left side and buffered rows from right side.
-   */
-  private def genScanner(ctx: CodegenContext): (String, String) = {
-    // Create class member for next row from both sides.
-    val leftRow = ctx.freshName("leftRow")
-    ctx.addMutableState("InternalRow", leftRow, "")
-    val rightRow = ctx.freshName("rightRow")
-    ctx.addMutableState("InternalRow", rightRow, s"$rightRow = null;")
-
-    // Create variables for join keys from both sides.
-    val leftKeyVars = createJoinKey(ctx, leftRow, leftKeys, left.output)
-    val leftAnyNull = leftKeyVars.map(_.isNull).mkString(" || ")
-    val rightKeyTmpVars = createJoinKey(ctx, rightRow, rightKeys, right.output)
-    val rightAnyNull = rightKeyTmpVars.map(_.isNull).mkString(" || ")
-    // Copy the right key as class members so they could be used in next function call.
-    val rightKeyVars = copyKeys(ctx, rightKeyTmpVars)
-
-    // A list to hold all matched rows from right side.
-    val matches = ctx.freshName("matches")
-    val clsName = classOf[java.util.ArrayList[InternalRow]].getName
-    ctx.addMutableState(clsName, matches, s"$matches = new $clsName();")
-    // Copy the left keys as class members so they could be used in next function call.
-    val matchedKeyVars = copyKeys(ctx, leftKeyVars)
-
-    ctx.addNewFunction("findNextInnerJoinRows",
-      s"""
-         |private boolean findNextInnerJoinRows(
-         |    scala.collection.Iterator leftIter,
-         |    scala.collection.Iterator rightIter) {
-         |  $leftRow = null;
-         |  int comp = 0;
-         |  while ($leftRow == null) {
-         |    if (!leftIter.hasNext()) return false;
-         |    $leftRow = (InternalRow) leftIter.next();
-         |    ${leftKeyVars.map(_.code).mkString("\n")}
-         |    if ($leftAnyNull) {
-         |      $leftRow = null;
-         |      continue;
-         |    }
-         |    if (!$matches.isEmpty()) {
-         |      ${genComparision(ctx, leftKeyVars, matchedKeyVars)}
-         |      if (comp == 0) {
-         |        return true;
-         |      }
-         |      $matches.clear();
-         |    }
-         |
-         |    do {
-         |      if ($rightRow == null) {
-         |        if (!rightIter.hasNext()) {
-         |          ${matchedKeyVars.map(_.code).mkString("\n")}
-         |          return !$matches.isEmpty();
-         |        }
-         |        $rightRow = (InternalRow) rightIter.next();
-         |        ${rightKeyTmpVars.map(_.code).mkString("\n")}
-         |        if ($rightAnyNull) {
-         |          $rightRow = null;
-         |          continue;
-         |        }
-         |        ${rightKeyVars.map(_.code).mkString("\n")}
-         |      }
-         |      ${genComparision(ctx, leftKeyVars, rightKeyVars)}
-         |      if (comp > 0) {
-         |        $rightRow = null;
-         |      } else if (comp < 0) {
-         |        if (!$matches.isEmpty()) {
-         |          ${matchedKeyVars.map(_.code).mkString("\n")}
-         |          return true;
-         |        }
-         |        $leftRow = null;
-         |      } else {
-         |        $matches.add($rightRow.copy());
-         |        $rightRow = null;;
-         |      }
-         |    } while ($leftRow != null);
-         |  }
-         |  return false; // unreachable
-         |}
-       """.stripMargin)
-
-    (leftRow, matches)
-  }
-
-  /**
-   * Creates variables for left part of result row.
-   *
-   * In order to defer the access after condition and also only access once in the loop,
-   * the variables should be declared separately from accessing the columns, we can't use the
-   * codegen of BoundReference here.
-   */
-  private def createLeftVars(ctx: CodegenContext, leftRow: String): Seq[ExprCode] = {
-    ctx.INPUT_ROW = leftRow
-    left.output.zipWithIndex.map { case (a, i) =>
-      val value = ctx.freshName("value")
-      val valueCode = ctx.getValue(leftRow, a.dataType, i.toString)
-      // declare it as class member, so we can access the column before or in the loop.
-      ctx.addMutableState(ctx.javaType(a.dataType), value, "")
-      if (a.nullable) {
-        val isNull = ctx.freshName("isNull")
-        ctx.addMutableState("boolean", isNull, "")
-        val code =
-          s"""
-             |$isNull = $leftRow.isNullAt($i);
-             |$value = $isNull ? ${ctx.defaultValue(a.dataType)} : ($valueCode);
-           """.stripMargin
-        ExprCode(code, isNull, value)
-      } else {
-        ExprCode(s"$value = $valueCode;", "false", value)
-      }
-    }
-  }
-
-  /**
-   * Creates the variables for right part of result row, using BoundReference, since the right
-   * part are accessed inside the loop.
-   */
-  private def createRightVar(ctx: CodegenContext, rightRow: String): Seq[ExprCode] = {
-    ctx.INPUT_ROW = rightRow
-    right.output.zipWithIndex.map { case (a, i) =>
-      BoundReference(i, a.dataType, a.nullable).genCode(ctx)
-    }
-  }
-
-  /**
-   * Splits variables based on whether it's used by condition or not, returns the code to create
-   * these variables before the condition and after the condition.
-   *
-   * Only a few columns are used by condition, then we can skip the accessing of those columns
-   * that are not used by condition also filtered out by condition.
-   */
-  private def splitVarsByCondition(
-      attributes: Seq[Attribute],
-      variables: Seq[ExprCode]): (String, String) = {
-    if (condition.isDefined) {
-      val condRefs = condition.get.references
-      val (used, notUsed) = attributes.zip(variables).partition{ case (a, ev) =>
-        condRefs.contains(a)
-      }
-      val beforeCond = evaluateVariables(used.map(_._2))
-      val afterCond = evaluateVariables(notUsed.map(_._2))
-      (beforeCond, afterCond)
-    } else {
-      (evaluateVariables(variables), "")
-    }
-  }
-
-  override def doProduce(ctx: CodegenContext): String = {
-    ctx.copyResult = true
-    val leftInput = ctx.freshName("leftInput")
-    ctx.addMutableState("scala.collection.Iterator", leftInput, s"$leftInput = inputs[0];")
-    val rightInput = ctx.freshName("rightInput")
-    ctx.addMutableState("scala.collection.Iterator", rightInput, s"$rightInput = inputs[1];")
-
-    val (leftRow, matches) = genScanner(ctx)
-
-    // Create variables for row from both sides.
-    val leftVars = createLeftVars(ctx, leftRow)
-    val rightRow = ctx.freshName("rightRow")
-    val rightVars = createRightVar(ctx, rightRow)
-
-    val size = ctx.freshName("size")
-    val i = ctx.freshName("i")
-    val numOutput = metricTerm(ctx, "numOutputRows")
-    val (beforeLoop, condCheck) = if (condition.isDefined) {
-      // Split the code of creating variables based on whether it's used by condition or not.
-      val loaded = ctx.freshName("loaded")
-      val (leftBefore, leftAfter) = splitVarsByCondition(left.output, leftVars)
-      val (rightBefore, rightAfter) = splitVarsByCondition(right.output, rightVars)
-      // Generate code for condition
-      ctx.currentVars = leftVars ++ rightVars
-      val cond = BindReferences.bindReference(condition.get, output).genCode(ctx)
-      // evaluate the columns those used by condition before loop
-      val before = s"""
-           |boolean $loaded = false;
-           |$leftBefore
-         """.stripMargin
-
-      val checking = s"""
-         |$rightBefore
-         |${cond.code}
-         |if (${cond.isNull} || !${cond.value}) continue;
-         |if (!$loaded) {
-         |  $loaded = true;
-         |  $leftAfter
-         |}
-         |$rightAfter
-     """.stripMargin
-      (before, checking)
-    } else {
-      (evaluateVariables(leftVars), "")
-    }
-
-    s"""
-       |while (findNextInnerJoinRows($leftInput, $rightInput)) {
-       |  int $size = $matches.size();
-       |  ${beforeLoop.trim}
-       |  for (int $i = 0; $i < $size; $i ++) {
-       |    InternalRow $rightRow = (InternalRow) $matches.get($i);
-       |    ${condCheck.trim}
-       |    $numOutput.add(1);
-       |    ${consume(ctx, leftVars ++ rightVars)}
-       |  }
-       |  if (shouldStop()) return;
-       |}
-     """.stripMargin
-  }
-}
-
-/**
- * Helper class that is used to implement [[SortMergeJoin]].
- *
- * To perform an inner (outer) join, users of this class call [[findNextInnerJoinRows()]]
- * ([[findNextOuterJoinRows()]]), which returns `true` if a result has been produced and `false`
- * otherwise. If a result has been produced, then the caller may call [[getStreamedRow]] to return
- * the matching row from the streamed input and may call [[getBufferedMatches]] to return the
- * sequence of matching rows from the buffered input (in the case of an outer join, this will return
- * an empty sequence if there are no matches from the buffered input). For efficiency, both of these
- * methods return mutable objects which are re-used across calls to the `findNext*JoinRows()`
- * methods.
- *
- * @param streamedKeyGenerator a projection that produces join keys from the streamed input.
- * @param bufferedKeyGenerator a projection that produces join keys from the buffered input.
- * @param keyOrdering an ordering which can be used to compare join keys.
- * @param streamedIter an input whose rows will be streamed.
- * @param bufferedIter an input whose rows will be buffered to construct sequences of rows that
- *                     have the same join key.
- */
-private[joins] class SortMergeJoinScanner(
-    streamedKeyGenerator: Projection,
-    bufferedKeyGenerator: Projection,
-    keyOrdering: Ordering[InternalRow],
-    streamedIter: RowIterator,
-    bufferedIter: RowIterator) {
-  private[this] var streamedRow: InternalRow = _
-  private[this] var streamedRowKey: InternalRow = _
-  private[this] var bufferedRow: InternalRow = _
-  // Note: this is guaranteed to never have any null columns:
-  private[this] var bufferedRowKey: InternalRow = _
-  /**
-   * The join key for the rows buffered in `bufferedMatches`, or null if `bufferedMatches` is empty
-   */
-  private[this] var matchJoinKey: InternalRow = _
-  /** Buffered rows from the buffered side of the join. This is empty if there are no matches. */
-  private[this] val bufferedMatches: ArrayBuffer[InternalRow] = new ArrayBuffer[InternalRow]
-
-  // Initialization (note: do _not_ want to advance streamed here).
-  advancedBufferedToRowWithNullFreeJoinKey()
-
-  // --- Public methods ---------------------------------------------------------------------------
-
-  def getStreamedRow: InternalRow = streamedRow
-
-  def getBufferedMatches: ArrayBuffer[InternalRow] = bufferedMatches
-
-  /**
-   * Advances both input iterators, stopping when we have found rows with matching join keys.
-   * @return true if matching rows have been found and false otherwise. If this returns true, then
-   *         [[getStreamedRow]] and [[getBufferedMatches]] can be called to construct the join
-   *         results.
-   */
-  final def findNextInnerJoinRows(): Boolean = {
-    while (advancedStreamed() && streamedRowKey.anyNull) {
-      // Advance the streamed side of the join until we find the next row whose join key contains
-      // no nulls or we hit the end of the streamed iterator.
-    }
-    if (streamedRow == null) {
-      // We have consumed the entire streamed iterator, so there can be no more matches.
-      matchJoinKey = null
-      bufferedMatches.clear()
-      false
-    } else if (matchJoinKey != null && keyOrdering.compare(streamedRowKey, matchJoinKey) == 0) {
-      // The new streamed row has the same join key as the previous row, so return the same matches.
-      true
-    } else if (bufferedRow == null) {
-      // The streamed row's join key does not match the current batch of buffered rows and there are
-      // no more rows to read from the buffered iterator, so there can be no more matches.
-      matchJoinKey = null
-      bufferedMatches.clear()
-      false
-    } else {
-      // Advance both the streamed and buffered iterators to find the next pair of matching rows.
-      var comp = keyOrdering.compare(streamedRowKey, bufferedRowKey)
-      do {
-        if (streamedRowKey.anyNull) {
-          advancedStreamed()
-        } else {
-          assert(!bufferedRowKey.anyNull)
-          comp = keyOrdering.compare(streamedRowKey, bufferedRowKey)
-          if (comp > 0) advancedBufferedToRowWithNullFreeJoinKey()
-          else if (comp < 0) advancedStreamed()
-        }
-      } while (streamedRow != null && bufferedRow != null && comp != 0)
-      if (streamedRow == null || bufferedRow == null) {
-        // We have either hit the end of one of the iterators, so there can be no more matches.
-        matchJoinKey = null
-        bufferedMatches.clear()
-        false
-      } else {
-        // The streamed row's join key matches the current buffered row's join, so walk through the
-        // buffered iterator to buffer the rest of the matching rows.
-        assert(comp == 0)
-        bufferMatchingRows()
-        true
-      }
-    }
-  }
-
-  /**
-   * Advances the streamed input iterator and buffers all rows from the buffered input that
-   * have matching keys.
-   * @return true if the streamed iterator returned a row, false otherwise. If this returns true,
-   *         then [[getStreamedRow]] and [[getBufferedMatches]] can be called to produce the outer
-   *         join results.
-   */
-  final def findNextOuterJoinRows(): Boolean = {
-    if (!advancedStreamed()) {
-      // We have consumed the entire streamed iterator, so there can be no more matches.
-      matchJoinKey = null
-      bufferedMatches.clear()
-      false
-    } else {
-      if (matchJoinKey != null && keyOrdering.compare(streamedRowKey, matchJoinKey) == 0) {
-        // Matches the current group, so do nothing.
-      } else {
-        // The streamed row does not match the current group.
-        matchJoinKey = null
-        bufferedMatches.clear()
-        if (bufferedRow != null && !streamedRowKey.anyNull) {
-          // The buffered iterator could still contain matching rows, so we'll need to walk through
-          // it until we either find matches or pass where they would be found.
-          var comp = 1
-          do {
-            comp = keyOrdering.compare(streamedRowKey, bufferedRowKey)
-          } while (comp > 0 && advancedBufferedToRowWithNullFreeJoinKey())
-          if (comp == 0) {
-            // We have found matches, so buffer them (this updates matchJoinKey)
-            bufferMatchingRows()
-          } else {
-            // We have overshot the position where the row would be found, hence no matches.
-          }
-        }
-      }
-      // If there is a streamed input then we always return true
-      true
-    }
-  }
-
-  // --- Private methods --------------------------------------------------------------------------
-
-  /**
-   * Advance the streamed iterator and compute the new row's join key.
-   * @return true if the streamed iterator returned a row and false otherwise.
-   */
-  private def advancedStreamed(): Boolean = {
-    if (streamedIter.advanceNext()) {
-      streamedRow = streamedIter.getRow
-      streamedRowKey = streamedKeyGenerator(streamedRow)
-      true
-    } else {
-      streamedRow = null
-      streamedRowKey = null
-      false
-    }
-  }
-
-  /**
-   * Advance the buffered iterator until we find a row with join key that does not contain nulls.
-   * @return true if the buffered iterator returned a row and false otherwise.
-   */
-  private def advancedBufferedToRowWithNullFreeJoinKey(): Boolean = {
-    var foundRow: Boolean = false
-    while (!foundRow && bufferedIter.advanceNext()) {
-      bufferedRow = bufferedIter.getRow
-      bufferedRowKey = bufferedKeyGenerator(bufferedRow)
-      foundRow = !bufferedRowKey.anyNull
-    }
-    if (!foundRow) {
-      bufferedRow = null
-      bufferedRowKey = null
-      false
-    } else {
-      true
-    }
-  }
-
-  /**
-   * Called when the streamed and buffered join keys match in order to buffer the matching rows.
-   */
-  private def bufferMatchingRows(): Unit = {
-    assert(streamedRowKey != null)
-    assert(!streamedRowKey.anyNull)
-    assert(bufferedRowKey != null)
-    assert(!bufferedRowKey.anyNull)
-    assert(keyOrdering.compare(streamedRowKey, bufferedRowKey) == 0)
-    // This join key may have been produced by a mutable projection, so we need to make a copy:
-    matchJoinKey = streamedRowKey.copy()
-    bufferedMatches.clear()
-    do {
-      bufferedMatches += bufferedRow.copy() // need to copy mutable rows before buffering them
-      advancedBufferedToRowWithNullFreeJoinKey()
-    } while (bufferedRow != null && keyOrdering.compare(streamedRowKey, bufferedRowKey) == 0)
-  }
-}
-
-/**
- * An iterator for outputting rows in left outer join.
- */
-private class LeftOuterIterator(
-    smjScanner: SortMergeJoinScanner,
-    rightNullRow: InternalRow,
-    boundCondition: InternalRow => Boolean,
-    resultProj: InternalRow => InternalRow,
-    numOutputRows: LongSQLMetric)
-  extends OneSideOuterIterator(
-    smjScanner, rightNullRow, boundCondition, resultProj, numOutputRows) {
-
-  protected override def setStreamSideOutput(row: InternalRow): Unit = joinedRow.withLeft(row)
-  protected override def setBufferedSideOutput(row: InternalRow): Unit = joinedRow.withRight(row)
-}
-
-/**
- * An iterator for outputting rows in right outer join.
- */
-private class RightOuterIterator(
-    smjScanner: SortMergeJoinScanner,
-    leftNullRow: InternalRow,
-    boundCondition: InternalRow => Boolean,
-    resultProj: InternalRow => InternalRow,
-    numOutputRows: LongSQLMetric)
-  extends OneSideOuterIterator(smjScanner, leftNullRow, boundCondition, resultProj, numOutputRows) {
-
-  protected override def setStreamSideOutput(row: InternalRow): Unit = joinedRow.withRight(row)
-  protected override def setBufferedSideOutput(row: InternalRow): Unit = joinedRow.withLeft(row)
-}
-
-/**
- * An abstract iterator for sharing code between [[LeftOuterIterator]] and [[RightOuterIterator]].
- *
- * Each [[OneSideOuterIterator]] has a streamed side and a buffered side. Each row on the
- * streamed side will output 0 or many rows, one for each matching row on the buffered side.
- * If there are no matches, then the buffered side of the joined output will be a null row.
- *
- * In left outer join, the left is the streamed side and the right is the buffered side.
- * In right outer join, the right is the streamed side and the left is the buffered side.
- *
- * @param smjScanner a scanner that streams rows and buffers any matching rows
- * @param bufferedSideNullRow the default row to return when a streamed row has no matches
- * @param boundCondition an additional filter condition for buffered rows
- * @param resultProj how the output should be projected
- * @param numOutputRows an accumulator metric for the number of rows output
- */
-private abstract class OneSideOuterIterator(
-    smjScanner: SortMergeJoinScanner,
-    bufferedSideNullRow: InternalRow,
-    boundCondition: InternalRow => Boolean,
-    resultProj: InternalRow => InternalRow,
-    numOutputRows: LongSQLMetric) extends RowIterator {
-
-  // A row to store the joined result, reused many times
-  protected[this] val joinedRow: JoinedRow = new JoinedRow()
-
-  // Index of the buffered rows, reset to 0 whenever we advance to a new streamed row
-  private[this] var bufferIndex: Int = 0
-
-  // This iterator is initialized lazily so there should be no matches initially
-  assert(smjScanner.getBufferedMatches.length == 0)
-
-  // Set output methods to be overridden by subclasses
-  protected def setStreamSideOutput(row: InternalRow): Unit
-  protected def setBufferedSideOutput(row: InternalRow): Unit
-
-  /**
-   * Advance to the next row on the stream side and populate the buffer with matches.
-   * @return whether there are more rows in the stream to consume.
-   */
-  private def advanceStream(): Boolean = {
-    bufferIndex = 0
-    if (smjScanner.findNextOuterJoinRows()) {
-      setStreamSideOutput(smjScanner.getStreamedRow)
-      if (smjScanner.getBufferedMatches.isEmpty) {
-        // There are no matching rows in the buffer, so return the null row
-        setBufferedSideOutput(bufferedSideNullRow)
-      } else {
-        // Find the next row in the buffer that satisfied the bound condition
-        if (!advanceBufferUntilBoundConditionSatisfied()) {
-          setBufferedSideOutput(bufferedSideNullRow)
-        }
-      }
-      true
-    } else {
-      // Stream has been exhausted
-      false
-    }
-  }
-
-  /**
-   * Advance to the next row in the buffer that satisfies the bound condition.
-   * @return whether there is such a row in the current buffer.
-   */
-  private def advanceBufferUntilBoundConditionSatisfied(): Boolean = {
-    var foundMatch: Boolean = false
-    while (!foundMatch && bufferIndex < smjScanner.getBufferedMatches.length) {
-      setBufferedSideOutput(smjScanner.getBufferedMatches(bufferIndex))
-      foundMatch = boundCondition(joinedRow)
-      bufferIndex += 1
-    }
-    foundMatch
-  }
-
-  override def advanceNext(): Boolean = {
-    val r = advanceBufferUntilBoundConditionSatisfied() || advanceStream()
-    if (r) numOutputRows += 1
-    r
-  }
-
-  override def getRow: InternalRow = resultProj(joinedRow)
-}
-
-private class SortMergeFullOuterJoinScanner(
-    leftKeyGenerator: Projection,
-    rightKeyGenerator: Projection,
-    keyOrdering: Ordering[InternalRow],
-    leftIter: RowIterator,
-    rightIter: RowIterator,
-    boundCondition: InternalRow => Boolean,
-    leftNullRow: InternalRow,
-    rightNullRow: InternalRow)  {
-  private[this] val joinedRow: JoinedRow = new JoinedRow()
-  private[this] var leftRow: InternalRow = _
-  private[this] var leftRowKey: InternalRow = _
-  private[this] var rightRow: InternalRow = _
-  private[this] var rightRowKey: InternalRow = _
-
-  private[this] var leftIndex: Int = 0
-  private[this] var rightIndex: Int = 0
-  private[this] val leftMatches: ArrayBuffer[InternalRow] = new ArrayBuffer[InternalRow]
-  private[this] val rightMatches: ArrayBuffer[InternalRow] = new ArrayBuffer[InternalRow]
-  private[this] var leftMatched: BitSet = new BitSet(1)
-  private[this] var rightMatched: BitSet = new BitSet(1)
-
-  advancedLeft()
-  advancedRight()
-
-  // --- Private methods --------------------------------------------------------------------------
-
-  /**
-   * Advance the left iterator and compute the new row's join key.
-   * @return true if the left iterator returned a row and false otherwise.
-   */
-  private def advancedLeft(): Boolean = {
-    if (leftIter.advanceNext()) {
-      leftRow = leftIter.getRow
-      leftRowKey = leftKeyGenerator(leftRow)
-      true
-    } else {
-      leftRow = null
-      leftRowKey = null
-      false
-    }
-  }
-
-  /**
-   * Advance the right iterator and compute the new row's join key.
-   * @return true if the right iterator returned a row and false otherwise.
-   */
-  private def advancedRight(): Boolean = {
-    if (rightIter.advanceNext()) {
-      rightRow = rightIter.getRow
-      rightRowKey = rightKeyGenerator(rightRow)
-      true
-    } else {
-      rightRow = null
-      rightRowKey = null
-      false
-    }
-  }
-
-  /**
-   * Populate the left and right buffers with rows matching the provided key.
-   * This consumes rows from both iterators until their keys are different from the matching key.
-   */
-  private def findMatchingRows(matchingKey: InternalRow): Unit = {
-    leftMatches.clear()
-    rightMatches.clear()
-    leftIndex = 0
-    rightIndex = 0
-
-    while (leftRowKey != null && keyOrdering.compare(leftRowKey, matchingKey) == 0) {
-      leftMatches += leftRow.copy()
-      advancedLeft()
-    }
-    while (rightRowKey != null && keyOrdering.compare(rightRowKey, matchingKey) == 0) {
-      rightMatches += rightRow.copy()
-      advancedRight()
-    }
-
-    if (leftMatches.size <= leftMatched.capacity) {
-      leftMatched.clear()
-    } else {
-      leftMatched = new BitSet(leftMatches.size)
-    }
-    if (rightMatches.size <= rightMatched.capacity) {
-      rightMatched.clear()
-    } else {
-      rightMatched = new BitSet(rightMatches.size)
-    }
-  }
-
-  /**
-   * Scan the left and right buffers for the next valid match.
-   *
-   * Note: this method mutates `joinedRow` to point to the latest matching rows in the buffers.
-   * If a left row has no valid matches on the right, or a right row has no valid matches on the
-   * left, then the row is joined with the null row and the result is considered a valid match.
-   *
-   * @return true if a valid match is found, false otherwise.
-   */
-  private def scanNextInBuffered(): Boolean = {
-    while (leftIndex < leftMatches.size) {
-      while (rightIndex < rightMatches.size) {
-        joinedRow(leftMatches(leftIndex), rightMatches(rightIndex))
-        if (boundCondition(joinedRow)) {
-          leftMatched.set(leftIndex)
-          rightMatched.set(rightIndex)
-          rightIndex += 1
-          return true
-        }
-        rightIndex += 1
-      }
-      rightIndex = 0
-      if (!leftMatched.get(leftIndex)) {
-        // the left row has never matched any right row, join it with null row
-        joinedRow(leftMatches(leftIndex), rightNullRow)
-        leftIndex += 1
-        return true
-      }
-      leftIndex += 1
-    }
-
-    while (rightIndex < rightMatches.size) {
-      if (!rightMatched.get(rightIndex)) {
-        // the right row has never matched any left row, join it with null row
-        joinedRow(leftNullRow, rightMatches(rightIndex))
-        rightIndex += 1
-        return true
-      }
-      rightIndex += 1
-    }
-
-    // There are no more valid matches in the left and right buffers
-    false
-  }
-
-  // --- Public methods --------------------------------------------------------------------------
-
-  def getJoinedRow(): JoinedRow = joinedRow
-
-  def advanceNext(): Boolean = {
-    // If we already buffered some matching rows, use them directly
-    if (leftIndex <= leftMatches.size || rightIndex <= rightMatches.size) {
-      if (scanNextInBuffered()) {
-        return true
-      }
-    }
-
-    if (leftRow != null && (leftRowKey.anyNull || rightRow == null)) {
-      joinedRow(leftRow.copy(), rightNullRow)
-      advancedLeft()
-      true
-    } else if (rightRow != null && (rightRowKey.anyNull || leftRow == null)) {
-      joinedRow(leftNullRow, rightRow.copy())
-      advancedRight()
-      true
-    } else if (leftRow != null && rightRow != null) {
-      // Both rows are present and neither have null values,
-      // so we populate the buffers with rows matching the next key
-      val comp = keyOrdering.compare(leftRowKey, rightRowKey)
-      if (comp <= 0) {
-        findMatchingRows(leftRowKey.copy())
-      } else {
-        findMatchingRows(rightRowKey.copy())
-      }
-      scanNextInBuffered()
-      true
-    } else {
-      // Both iterators have been consumed
-      false
-    }
-  }
-}
-
-private class FullOuterIterator(
-    smjScanner: SortMergeFullOuterJoinScanner,
-    resultProj: InternalRow => InternalRow,
-    numRows: LongSQLMetric) extends RowIterator {
-  private[this] val joinedRow: JoinedRow = smjScanner.getJoinedRow()
-
-  override def advanceNext(): Boolean = {
-    val r = smjScanner.advanceNext()
-    if (r) numRows += 1
-    r
-  }
-
-  override def getRow: InternalRow = resultProj(joinedRow)
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
new file mode 100644
index 0000000..96b283a
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
@@ -0,0 +1,964 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.joins
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.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.{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 SortMergeJoinExec(
+    leftKeys: Seq[Expression],
+    rightKeys: Seq[Expression],
+    joinType: JoinType,
+    condition: Option[Expression],
+    left: SparkPlan,
+    right: SparkPlan) extends BinaryExecNode with CodegenSupport {
+
+  override private[sql] lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
+
+  override def output: Seq[Attribute] = {
+    joinType match {
+      case Inner =>
+        left.output ++ right.output
+      case LeftOuter =>
+        left.output ++ right.output.map(_.withNullability(true))
+      case RightOuter =>
+        left.output.map(_.withNullability(true)) ++ right.output
+      case FullOuter =>
+        (left.output ++ right.output).map(_.withNullability(true))
+      case x =>
+        throw new IllegalArgumentException(
+          s"${getClass.getSimpleName} should not take $x as the JoinType")
+    }
+  }
+
+  override def outputPartitioning: Partitioning = joinType match {
+    case Inner => PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning))
+    // For left and right outer joins, the output is partitioned by the streamed input's join keys.
+    case LeftOuter => left.outputPartitioning
+    case RightOuter => right.outputPartitioning
+    case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions)
+    case x =>
+      throw new IllegalArgumentException(
+        s"${getClass.getSimpleName} should not take $x as the JoinType")
+  }
+
+  override def requiredChildDistribution: Seq[Distribution] =
+    ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil
+
+  override def outputOrdering: Seq[SortOrder] = requiredOrders(leftKeys)
+
+  override def requiredChildOrdering: Seq[Seq[SortOrder]] =
+    requiredOrders(leftKeys) :: requiredOrders(rightKeys) :: Nil
+
+  private def requiredOrders(keys: Seq[Expression]): Seq[SortOrder] = {
+    // This must be ascending in order to agree with the `keyOrdering` defined in `doExecute()`.
+    keys.map(SortOrder(_, Ascending))
+  }
+
+  private def createLeftKeyGenerator(): Projection =
+    UnsafeProjection.create(leftKeys, left.output)
+
+  private def createRightKeyGenerator(): Projection =
+    UnsafeProjection.create(rightKeys, right.output)
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    val numOutputRows = longMetric("numOutputRows")
+
+    left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) =>
+      val boundCondition: (InternalRow) => Boolean = {
+        condition.map { cond =>
+          newPredicate(cond, left.output ++ right.output)
+        }.getOrElse {
+          (r: InternalRow) => true
+        }
+      }
+      // An ordering that can be used to compare keys from both sides.
+      val keyOrdering = newNaturalAscendingOrdering(leftKeys.map(_.dataType))
+      val resultProj: InternalRow => InternalRow = UnsafeProjection.create(output, output)
+
+      joinType match {
+        case Inner =>
+          new RowIterator {
+            // The projection used to extract keys from input rows of the left child.
+            private[this] val leftKeyGenerator = UnsafeProjection.create(leftKeys, left.output)
+
+            // The projection used to extract keys from input rows of the right child.
+            private[this] val rightKeyGenerator = UnsafeProjection.create(rightKeys, right.output)
+
+            // An ordering that can be used to compare keys from both sides.
+            private[this] val keyOrdering = newNaturalAscendingOrdering(leftKeys.map(_.dataType))
+            private[this] var currentLeftRow: InternalRow = _
+            private[this] var currentRightMatches: ArrayBuffer[InternalRow] = _
+            private[this] var currentMatchIdx: Int = -1
+            private[this] val smjScanner = new SortMergeJoinScanner(
+              leftKeyGenerator,
+              rightKeyGenerator,
+              keyOrdering,
+              RowIterator.fromScala(leftIter),
+              RowIterator.fromScala(rightIter)
+            )
+            private[this] val joinRow = new JoinedRow
+            private[this] val resultProjection: (InternalRow) => InternalRow =
+              UnsafeProjection.create(schema)
+
+            if (smjScanner.findNextInnerJoinRows()) {
+              currentRightMatches = smjScanner.getBufferedMatches
+              currentLeftRow = smjScanner.getStreamedRow
+              currentMatchIdx = 0
+            }
+
+            override def advanceNext(): Boolean = {
+              while (currentMatchIdx >= 0) {
+                if (currentMatchIdx == currentRightMatches.length) {
+                  if (smjScanner.findNextInnerJoinRows()) {
+                    currentRightMatches = smjScanner.getBufferedMatches
+                    currentLeftRow = smjScanner.getStreamedRow
+                    currentMatchIdx = 0
+                  } else {
+                    currentRightMatches = null
+                    currentLeftRow = null
+                    currentMatchIdx = -1
+                    return false
+                  }
+                }
+                joinRow(currentLeftRow, currentRightMatches(currentMatchIdx))
+                currentMatchIdx += 1
+                if (boundCondition(joinRow)) {
+                  numOutputRows += 1
+                  return true
+                }
+              }
+              false
+            }
+
+            override def getRow: InternalRow = resultProjection(joinRow)
+          }.toScala
+
+        case LeftOuter =>
+          val smjScanner = new SortMergeJoinScanner(
+            streamedKeyGenerator = createLeftKeyGenerator(),
+            bufferedKeyGenerator = createRightKeyGenerator(),
+            keyOrdering,
+            streamedIter = RowIterator.fromScala(leftIter),
+            bufferedIter = RowIterator.fromScala(rightIter)
+          )
+          val rightNullRow = new GenericInternalRow(right.output.length)
+          new LeftOuterIterator(
+            smjScanner, rightNullRow, boundCondition, resultProj, numOutputRows).toScala
+
+        case RightOuter =>
+          val smjScanner = new SortMergeJoinScanner(
+            streamedKeyGenerator = createRightKeyGenerator(),
+            bufferedKeyGenerator = createLeftKeyGenerator(),
+            keyOrdering,
+            streamedIter = RowIterator.fromScala(rightIter),
+            bufferedIter = RowIterator.fromScala(leftIter)
+          )
+          val leftNullRow = new GenericInternalRow(left.output.length)
+          new RightOuterIterator(
+            smjScanner, leftNullRow, boundCondition, resultProj, numOutputRows).toScala
+
+        case FullOuter =>
+          val leftNullRow = new GenericInternalRow(left.output.length)
+          val rightNullRow = new GenericInternalRow(right.output.length)
+          val smjScanner = new SortMergeFullOuterJoinScanner(
+            leftKeyGenerator = createLeftKeyGenerator(),
+            rightKeyGenerator = createRightKeyGenerator(),
+            keyOrdering,
+            leftIter = RowIterator.fromScala(leftIter),
+            rightIter = RowIterator.fromScala(rightIter),
+            boundCondition,
+            leftNullRow,
+            rightNullRow)
+
+          new FullOuterIterator(
+            smjScanner,
+            resultProj,
+            numOutputRows).toScala
+
+        case x =>
+          throw new IllegalArgumentException(
+            s"SortMergeJoin should not take $x as the JoinType")
+      }
+
+    }
+  }
+
+  override def supportCodegen: Boolean = {
+    joinType == Inner
+  }
+
+  override def inputRDDs(): Seq[RDD[InternalRow]] = {
+    left.execute() :: right.execute() :: Nil
+  }
+
+  private def createJoinKey(
+      ctx: CodegenContext,
+      row: String,
+      keys: Seq[Expression],
+      input: Seq[Attribute]): Seq[ExprCode] = {
+    ctx.INPUT_ROW = row
+    keys.map(BindReferences.bindReference(_, input).genCode(ctx))
+  }
+
+  private def copyKeys(ctx: CodegenContext, vars: Seq[ExprCode]): Seq[ExprCode] = {
+    vars.zipWithIndex.map { case (ev, i) =>
+      val value = ctx.freshName("value")
+      ctx.addMutableState(ctx.javaType(leftKeys(i).dataType), value, "")
+      val code =
+        s"""
+           |$value = ${ev.value};
+         """.stripMargin
+      ExprCode(code, "false", value)
+    }
+  }
+
+  private def genComparision(ctx: CodegenContext, a: Seq[ExprCode], b: Seq[ExprCode]): String = {
+    val comparisons = a.zip(b).zipWithIndex.map { case ((l, r), i) =>
+      s"""
+         |if (comp == 0) {
+         |  comp = ${ctx.genComp(leftKeys(i).dataType, l.value, r.value)};
+         |}
+       """.stripMargin.trim
+    }
+    s"""
+       |comp = 0;
+       |${comparisons.mkString("\n")}
+     """.stripMargin
+  }
+
+  /**
+   * Generate a function to scan both left and right to find a match, returns the term for
+   * matched one row from left side and buffered rows from right side.
+   */
+  private def genScanner(ctx: CodegenContext): (String, String) = {
+    // Create class member for next row from both sides.
+    val leftRow = ctx.freshName("leftRow")
+    ctx.addMutableState("InternalRow", leftRow, "")
+    val rightRow = ctx.freshName("rightRow")
+    ctx.addMutableState("InternalRow", rightRow, s"$rightRow = null;")
+
+    // Create variables for join keys from both sides.
+    val leftKeyVars = createJoinKey(ctx, leftRow, leftKeys, left.output)
+    val leftAnyNull = leftKeyVars.map(_.isNull).mkString(" || ")
+    val rightKeyTmpVars = createJoinKey(ctx, rightRow, rightKeys, right.output)
+    val rightAnyNull = rightKeyTmpVars.map(_.isNull).mkString(" || ")
+    // Copy the right key as class members so they could be used in next function call.
+    val rightKeyVars = copyKeys(ctx, rightKeyTmpVars)
+
+    // A list to hold all matched rows from right side.
+    val matches = ctx.freshName("matches")
+    val clsName = classOf[java.util.ArrayList[InternalRow]].getName
+    ctx.addMutableState(clsName, matches, s"$matches = new $clsName();")
+    // Copy the left keys as class members so they could be used in next function call.
+    val matchedKeyVars = copyKeys(ctx, leftKeyVars)
+
+    ctx.addNewFunction("findNextInnerJoinRows",
+      s"""
+         |private boolean findNextInnerJoinRows(
+         |    scala.collection.Iterator leftIter,
+         |    scala.collection.Iterator rightIter) {
+         |  $leftRow = null;
+         |  int comp = 0;
+         |  while ($leftRow == null) {
+         |    if (!leftIter.hasNext()) return false;
+         |    $leftRow = (InternalRow) leftIter.next();
+         |    ${leftKeyVars.map(_.code).mkString("\n")}
+         |    if ($leftAnyNull) {
+         |      $leftRow = null;
+         |      continue;
+         |    }
+         |    if (!$matches.isEmpty()) {
+         |      ${genComparision(ctx, leftKeyVars, matchedKeyVars)}
+         |      if (comp == 0) {
+         |        return true;
+         |      }
+         |      $matches.clear();
+         |    }
+         |
+         |    do {
+         |      if ($rightRow == null) {
+         |        if (!rightIter.hasNext()) {
+         |          ${matchedKeyVars.map(_.code).mkString("\n")}
+         |          return !$matches.isEmpty();
+         |        }
+         |        $rightRow = (InternalRow) rightIter.next();
+         |        ${rightKeyTmpVars.map(_.code).mkString("\n")}
+         |        if ($rightAnyNull) {
+         |          $rightRow = null;
+         |          continue;
+         |        }
+         |        ${rightKeyVars.map(_.code).mkString("\n")}
+         |      }
+         |      ${genComparision(ctx, leftKeyVars, rightKeyVars)}
+         |      if (comp > 0) {
+         |        $rightRow = null;
+         |      } else if (comp < 0) {
+         |        if (!$matches.isEmpty()) {
+         |          ${matchedKeyVars.map(_.code).mkString("\n")}
+         |          return true;
+         |        }
+         |        $leftRow = null;
+         |      } else {
+         |        $matches.add($rightRow.copy());
+         |        $rightRow = null;;
+         |      }
+         |    } while ($leftRow != null);
+         |  }
+         |  return false; // unreachable
+         |}
+       """.stripMargin)
+
+    (leftRow, matches)
+  }
+
+  /**
+   * Creates variables for left part of result row.
+   *
+   * In order to defer the access after condition and also only access once in the loop,
+   * the variables should be declared separately from accessing the columns, we can't use the
+   * codegen of BoundReference here.
+   */
+  private def createLeftVars(ctx: CodegenContext, leftRow: String): Seq[ExprCode] = {
+    ctx.INPUT_ROW = leftRow
+    left.output.zipWithIndex.map { case (a, i) =>
+      val value = ctx.freshName("value")
+      val valueCode = ctx.getValue(leftRow, a.dataType, i.toString)
+      // declare it as class member, so we can access the column before or in the loop.
+      ctx.addMutableState(ctx.javaType(a.dataType), value, "")
+      if (a.nullable) {
+        val isNull = ctx.freshName("isNull")
+        ctx.addMutableState("boolean", isNull, "")
+        val code =
+          s"""
+             |$isNull = $leftRow.isNullAt($i);
+             |$value = $isNull ? ${ctx.defaultValue(a.dataType)} : ($valueCode);
+           """.stripMargin
+        ExprCode(code, isNull, value)
+      } else {
+        ExprCode(s"$value = $valueCode;", "false", value)
+      }
+    }
+  }
+
+  /**
+   * Creates the variables for right part of result row, using BoundReference, since the right
+   * part are accessed inside the loop.
+   */
+  private def createRightVar(ctx: CodegenContext, rightRow: String): Seq[ExprCode] = {
+    ctx.INPUT_ROW = rightRow
+    right.output.zipWithIndex.map { case (a, i) =>
+      BoundReference(i, a.dataType, a.nullable).genCode(ctx)
+    }
+  }
+
+  /**
+   * Splits variables based on whether it's used by condition or not, returns the code to create
+   * these variables before the condition and after the condition.
+   *
+   * Only a few columns are used by condition, then we can skip the accessing of those columns
+   * that are not used by condition also filtered out by condition.
+   */
+  private def splitVarsByCondition(
+      attributes: Seq[Attribute],
+      variables: Seq[ExprCode]): (String, String) = {
+    if (condition.isDefined) {
+      val condRefs = condition.get.references
+      val (used, notUsed) = attributes.zip(variables).partition{ case (a, ev) =>
+        condRefs.contains(a)
+      }
+      val beforeCond = evaluateVariables(used.map(_._2))
+      val afterCond = evaluateVariables(notUsed.map(_._2))
+      (beforeCond, afterCond)
+    } else {
+      (evaluateVariables(variables), "")
+    }
+  }
+
+  override def doProduce(ctx: CodegenContext): String = {
+    ctx.copyResult = true
+    val leftInput = ctx.freshName("leftInput")
+    ctx.addMutableState("scala.collection.Iterator", leftInput, s"$leftInput = inputs[0];")
+    val rightInput = ctx.freshName("rightInput")
+    ctx.addMutableState("scala.collection.Iterator", rightInput, s"$rightInput = inputs[1];")
+
+    val (leftRow, matches) = genScanner(ctx)
+
+    // Create variables for row from both sides.
+    val leftVars = createLeftVars(ctx, leftRow)
+    val rightRow = ctx.freshName("rightRow")
+    val rightVars = createRightVar(ctx, rightRow)
+
+    val size = ctx.freshName("size")
+    val i = ctx.freshName("i")
+    val numOutput = metricTerm(ctx, "numOutputRows")
+    val (beforeLoop, condCheck) = if (condition.isDefined) {
+      // Split the code of creating variables based on whether it's used by condition or not.
+      val loaded = ctx.freshName("loaded")
+      val (leftBefore, leftAfter) = splitVarsByCondition(left.output, leftVars)
+      val (rightBefore, rightAfter) = splitVarsByCondition(right.output, rightVars)
+      // Generate code for condition
+      ctx.currentVars = leftVars ++ rightVars
+      val cond = BindReferences.bindReference(condition.get, output).genCode(ctx)
+      // evaluate the columns those used by condition before loop
+      val before = s"""
+           |boolean $loaded = false;
+           |$leftBefore
+         """.stripMargin
+
+      val checking = s"""
+         |$rightBefore
+         |${cond.code}
+         |if (${cond.isNull} || !${cond.value}) continue;
+         |if (!$loaded) {
+         |  $loaded = true;
+         |  $leftAfter
+         |}
+         |$rightAfter
+     """.stripMargin
+      (before, checking)
+    } else {
+      (evaluateVariables(leftVars), "")
+    }
+
+    s"""
+       |while (findNextInnerJoinRows($leftInput, $rightInput)) {
+       |  int $size = $matches.size();
+       |  ${beforeLoop.trim}
+       |  for (int $i = 0; $i < $size; $i ++) {
+       |    InternalRow $rightRow = (InternalRow) $matches.get($i);
+       |    ${condCheck.trim}
+       |    $numOutput.add(1);
+       |    ${consume(ctx, leftVars ++ rightVars)}
+       |  }
+       |  if (shouldStop()) return;
+       |}
+     """.stripMargin
+  }
+}
+
+/**
+ * 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`
+ * otherwise. If a result has been produced, then the caller may call [[getStreamedRow]] to return
+ * the matching row from the streamed input and may call [[getBufferedMatches]] to return the
+ * sequence of matching rows from the buffered input (in the case of an outer join, this will return
+ * an empty sequence if there are no matches from the buffered input). For efficiency, both of these
+ * methods return mutable objects which are re-used across calls to the `findNext*JoinRows()`
+ * methods.
+ *
+ * @param streamedKeyGenerator a projection that produces join keys from the streamed input.
+ * @param bufferedKeyGenerator a projection that produces join keys from the buffered input.
+ * @param keyOrdering an ordering which can be used to compare join keys.
+ * @param streamedIter an input whose rows will be streamed.
+ * @param bufferedIter an input whose rows will be buffered to construct sequences of rows that
+ *                     have the same join key.
+ */
+private[joins] class SortMergeJoinScanner(
+    streamedKeyGenerator: Projection,
+    bufferedKeyGenerator: Projection,
+    keyOrdering: Ordering[InternalRow],
+    streamedIter: RowIterator,
+    bufferedIter: RowIterator) {
+  private[this] var streamedRow: InternalRow = _
+  private[this] var streamedRowKey: InternalRow = _
+  private[this] var bufferedRow: InternalRow = _
+  // Note: this is guaranteed to never have any null columns:
+  private[this] var bufferedRowKey: InternalRow = _
+  /**
+   * The join key for the rows buffered in `bufferedMatches`, or null if `bufferedMatches` is empty
+   */
+  private[this] var matchJoinKey: InternalRow = _
+  /** Buffered rows from the buffered side of the join. This is empty if there are no matches. */
+  private[this] val bufferedMatches: ArrayBuffer[InternalRow] = new ArrayBuffer[InternalRow]
+
+  // Initialization (note: do _not_ want to advance streamed here).
+  advancedBufferedToRowWithNullFreeJoinKey()
+
+  // --- Public methods ---------------------------------------------------------------------------
+
+  def getStreamedRow: InternalRow = streamedRow
+
+  def getBufferedMatches: ArrayBuffer[InternalRow] = bufferedMatches
+
+  /**
+   * Advances both input iterators, stopping when we have found rows with matching join keys.
+   * @return true if matching rows have been found and false otherwise. If this returns true, then
+   *         [[getStreamedRow]] and [[getBufferedMatches]] can be called to construct the join
+   *         results.
+   */
+  final def findNextInnerJoinRows(): Boolean = {
+    while (advancedStreamed() && streamedRowKey.anyNull) {
+      // Advance the streamed side of the join until we find the next row whose join key contains
+      // no nulls or we hit the end of the streamed iterator.
+    }
+    if (streamedRow == null) {
+      // We have consumed the entire streamed iterator, so there can be no more matches.
+      matchJoinKey = null
+      bufferedMatches.clear()
+      false
+    } else if (matchJoinKey != null && keyOrdering.compare(streamedRowKey, matchJoinKey) == 0) {
+      // The new streamed row has the same join key as the previous row, so return the same matches.
+      true
+    } else if (bufferedRow == null) {
+      // The streamed row's join key does not match the current batch of buffered rows and there are
+      // no more rows to read from the buffered iterator, so there can be no more matches.
+      matchJoinKey = null
+      bufferedMatches.clear()
+      false
+    } else {
+      // Advance both the streamed and buffered iterators to find the next pair of matching rows.
+      var comp = keyOrdering.compare(streamedRowKey, bufferedRowKey)
+      do {
+        if (streamedRowKey.anyNull) {
+          advancedStreamed()
+        } else {
+          assert(!bufferedRowKey.anyNull)
+          comp = keyOrdering.compare(streamedRowKey, bufferedRowKey)
+          if (comp > 0) advancedBufferedToRowWithNullFreeJoinKey()
+          else if (comp < 0) advancedStreamed()
+        }
+      } while (streamedRow != null && bufferedRow != null && comp != 0)
+      if (streamedRow == null || bufferedRow == null) {
+        // We have either hit the end of one of the iterators, so there can be no more matches.
+        matchJoinKey = null
+        bufferedMatches.clear()
+        false
+      } else {
+        // The streamed row's join key matches the current buffered row's join, so walk through the
+        // buffered iterator to buffer the rest of the matching rows.
+        assert(comp == 0)
+        bufferMatchingRows()
+        true
+      }
+    }
+  }
+
+  /**
+   * Advances the streamed input iterator and buffers all rows from the buffered input that
+   * have matching keys.
+   * @return true if the streamed iterator returned a row, false otherwise. If this returns true,
+   *         then [[getStreamedRow]] and [[getBufferedMatches]] can be called to produce the outer
+   *         join results.
+   */
+  final def findNextOuterJoinRows(): Boolean = {
+    if (!advancedStreamed()) {
+      // We have consumed the entire streamed iterator, so there can be no more matches.
+      matchJoinKey = null
+      bufferedMatches.clear()
+      false
+    } else {
+      if (matchJoinKey != null && keyOrdering.compare(streamedRowKey, matchJoinKey) == 0) {
+        // Matches the current group, so do nothing.
+      } else {
+        // The streamed row does not match the current group.
+        matchJoinKey = null
+        bufferedMatches.clear()
+        if (bufferedRow != null && !streamedRowKey.anyNull) {
+          // The buffered iterator could still contain matching rows, so we'll need to walk through
+          // it until we either find matches or pass where they would be found.
+          var comp = 1
+          do {
+            comp = keyOrdering.compare(streamedRowKey, bufferedRowKey)
+          } while (comp > 0 && advancedBufferedToRowWithNullFreeJoinKey())
+          if (comp == 0) {
+            // We have found matches, so buffer them (this updates matchJoinKey)
+            bufferMatchingRows()
+          } else {
+            // We have overshot the position where the row would be found, hence no matches.
+          }
+        }
+      }
+      // If there is a streamed input then we always return true
+      true
+    }
+  }
+
+  // --- Private methods --------------------------------------------------------------------------
+
+  /**
+   * Advance the streamed iterator and compute the new row's join key.
+   * @return true if the streamed iterator returned a row and false otherwise.
+   */
+  private def advancedStreamed(): Boolean = {
+    if (streamedIter.advanceNext()) {
+      streamedRow = streamedIter.getRow
+      streamedRowKey = streamedKeyGenerator(streamedRow)
+      true
+    } else {
+      streamedRow = null
+      streamedRowKey = null
+      false
+    }
+  }
+
+  /**
+   * Advance the buffered iterator until we find a row with join key that does not contain nulls.
+   * @return true if the buffered iterator returned a row and false otherwise.
+   */
+  private def advancedBufferedToRowWithNullFreeJoinKey(): Boolean = {
+    var foundRow: Boolean = false
+    while (!foundRow && bufferedIter.advanceNext()) {
+      bufferedRow = bufferedIter.getRow
+      bufferedRowKey = bufferedKeyGenerator(bufferedRow)
+      foundRow = !bufferedRowKey.anyNull
+    }
+    if (!foundRow) {
+      bufferedRow = null
+      bufferedRowKey = null
+      false
+    } else {
+      true
+    }
+  }
+
+  /**
+   * Called when the streamed and buffered join keys match in order to buffer the matching rows.
+   */
+  private def bufferMatchingRows(): Unit = {
+    assert(streamedRowKey != null)
+    assert(!streamedRowKey.anyNull)
+    assert(bufferedRowKey != null)
+    assert(!bufferedRowKey.anyNull)
+    assert(keyOrdering.compare(streamedRowKey, bufferedRowKey) == 0)
+    // This join key may have been produced by a mutable projection, so we need to make a copy:
+    matchJoinKey = streamedRowKey.copy()
+    bufferedMatches.clear()
+    do {
+      bufferedMatches += bufferedRow.copy() // need to copy mutable rows before buffering them
+      advancedBufferedToRowWithNullFreeJoinKey()
+    } while (bufferedRow != null && keyOrdering.compare(streamedRowKey, bufferedRowKey) == 0)
+  }
+}
+
+/**
+ * An iterator for outputting rows in left outer join.
+ */
+private class LeftOuterIterator(
+    smjScanner: SortMergeJoinScanner,
+    rightNullRow: InternalRow,
+    boundCondition: InternalRow => Boolean,
+    resultProj: InternalRow => InternalRow,
+    numOutputRows: LongSQLMetric)
+  extends OneSideOuterIterator(
+    smjScanner, rightNullRow, boundCondition, resultProj, numOutputRows) {
+
+  protected override def setStreamSideOutput(row: InternalRow): Unit = joinedRow.withLeft(row)
+  protected override def setBufferedSideOutput(row: InternalRow): Unit = joinedRow.withRight(row)
+}
+
+/**
+ * An iterator for outputting rows in right outer join.
+ */
+private class RightOuterIterator(
+    smjScanner: SortMergeJoinScanner,
+    leftNullRow: InternalRow,
+    boundCondition: InternalRow => Boolean,
+    resultProj: InternalRow => InternalRow,
+    numOutputRows: LongSQLMetric)
+  extends OneSideOuterIterator(smjScanner, leftNullRow, boundCondition, resultProj, numOutputRows) {
+
+  protected override def setStreamSideOutput(row: InternalRow): Unit = joinedRow.withRight(row)
+  protected override def setBufferedSideOutput(row: InternalRow): Unit = joinedRow.withLeft(row)
+}
+
+/**
+ * An abstract iterator for sharing code between [[LeftOuterIterator]] and [[RightOuterIterator]].
+ *
+ * Each [[OneSideOuterIterator]] has a streamed side and a buffered side. Each row on the
+ * streamed side will output 0 or many rows, one for each matching row on the buffered side.
+ * If there are no matches, then the buffered side of the joined output will be a null row.
+ *
+ * In left outer join, the left is the streamed side and the right is the buffered side.
+ * In right outer join, the right is the streamed side and the left is the buffered side.
+ *
+ * @param smjScanner a scanner that streams rows and buffers any matching rows
+ * @param bufferedSideNullRow the default row to return when a streamed row has no matches
+ * @param boundCondition an additional filter condition for buffered rows
+ * @param resultProj how the output should be projected
+ * @param numOutputRows an accumulator metric for the number of rows output
+ */
+private abstract class OneSideOuterIterator(
+    smjScanner: SortMergeJoinScanner,
+    bufferedSideNullRow: InternalRow,
+    boundCondition: InternalRow => Boolean,
+    resultProj: InternalRow => InternalRow,
+    numOutputRows: LongSQLMetric) extends RowIterator {
+
+  // A row to store the joined result, reused many times
+  protected[this] val joinedRow: JoinedRow = new JoinedRow()
+
+  // Index of the buffered rows, reset to 0 whenever we advance to a new streamed row
+  private[this] var bufferIndex: Int = 0
+
+  // This iterator is initialized lazily so there should be no matches initially
+  assert(smjScanner.getBufferedMatches.length == 0)
+
+  // Set output methods to be overridden by subclasses
+  protected def setStreamSideOutput(row: InternalRow): Unit
+  protected def setBufferedSideOutput(row: InternalRow): Unit
+
+  /**
+   * Advance to the next row on the stream side and populate the buffer with matches.
+   * @return whether there are more rows in the stream to consume.
+   */
+  private def advanceStream(): Boolean = {
+    bufferIndex = 0
+    if (smjScanner.findNextOuterJoinRows()) {
+      setStreamSideOutput(smjScanner.getStreamedRow)
+      if (smjScanner.getBufferedMatches.isEmpty) {
+        // There are no matching rows in the buffer, so return the null row
+        setBufferedSideOutput(bufferedSideNullRow)
+      } else {
+        // Find the next row in the buffer that satisfied the bound condition
+        if (!advanceBufferUntilBoundConditionSatisfied()) {
+          setBufferedSideOutput(bufferedSideNullRow)
+        }
+      }
+      true
+    } else {
+      // Stream has been exhausted
+      false
+    }
+  }
+
+  /**
+   * Advance to the next row in the buffer that satisfies the bound condition.
+   * @return whether there is such a row in the current buffer.
+   */
+  private def advanceBufferUntilBoundConditionSatisfied(): Boolean = {
+    var foundMatch: Boolean = false
+    while (!foundMatch && bufferIndex < smjScanner.getBufferedMatches.length) {
+      setBufferedSideOutput(smjScanner.getBufferedMatches(bufferIndex))
+      foundMatch = boundCondition(joinedRow)
+      bufferIndex += 1
+    }
+    foundMatch
+  }
+
+  override def advanceNext(): Boolean = {
+    val r = advanceBufferUntilBoundConditionSatisfied() || advanceStream()
+    if (r) numOutputRows += 1
+    r
+  }
+
+  override def getRow: InternalRow = resultProj(joinedRow)
+}
+
+private class SortMergeFullOuterJoinScanner(
+    leftKeyGenerator: Projection,
+    rightKeyGenerator: Projection,
+    keyOrdering: Ordering[InternalRow],
+    leftIter: RowIterator,
+    rightIter: RowIterator,
+    boundCondition: InternalRow => Boolean,
+    leftNullRow: InternalRow,
+    rightNullRow: InternalRow)  {
+  private[this] val joinedRow: JoinedRow = new JoinedRow()
+  private[this] var leftRow: InternalRow = _
+  private[this] var leftRowKey: InternalRow = _
+  private[this] var rightRow: InternalRow = _
+  private[this] var rightRowKey: InternalRow = _
+
+  private[this] var leftIndex: Int = 0
+  private[this] var rightIndex: Int = 0
+  private[this] val leftMatches: ArrayBuffer[InternalRow] = new ArrayBuffer[InternalRow]
+  private[this] val rightMatches: ArrayBuffer[InternalRow] = new ArrayBuffer[InternalRow]
+  private[this] var leftMatched: BitSet = new BitSet(1)
+  private[this] var rightMatched: BitSet = new BitSet(1)
+
+  advancedLeft()
+  advancedRight()
+
+  // --- Private methods --------------------------------------------------------------------------
+
+  /**
+   * Advance the left iterator and compute the new row's join key.
+   * @return true if the left iterator returned a row and false otherwise.
+   */
+  private def advancedLeft(): Boolean = {
+    if (leftIter.advanceNext()) {
+      leftRow = leftIter.getRow
+      leftRowKey = leftKeyGenerator(leftRow)
+      true
+    } else {
+      leftRow = null
+      leftRowKey = null
+      false
+    }
+  }
+
+  /**
+   * Advance the right iterator and compute the new row's join key.
+   * @return true if the right iterator returned a row and false otherwise.
+   */
+  private def advancedRight(): Boolean = {
+    if (rightIter.advanceNext()) {
+      rightRow = rightIter.getRow
+      rightRowKey = rightKeyGenerator(rightRow)
+      true
+    } else {
+      rightRow = null
+      rightRowKey = null
+      false
+    }
+  }
+
+  /**
+   * Populate the left and right buffers with rows matching the provided key.
+   * This consumes rows from both iterators until their keys are different from the matching key.
+   */
+  private def findMatchingRows(matchingKey: InternalRow): Unit = {
+    leftMatches.clear()
+    rightMatches.clear()
+    leftIndex = 0
+    rightIndex = 0
+
+    while (leftRowKey != null && keyOrdering.compare(leftRowKey, matchingKey) == 0) {
+      leftMatches += leftRow.copy()
+      advancedLeft()
+    }
+    while (rightRowKey != null && keyOrdering.compare(rightRowKey, matchingKey) == 0) {
+      rightMatches += rightRow.copy()
+      advancedRight()
+    }
+
+    if (leftMatches.size <= leftMatched.capacity) {
+      leftMatched.clear()
+    } else {
+      leftMatched = new BitSet(leftMatches.size)
+    }
+    if (rightMatches.size <= rightMatched.capacity) {
+      rightMatched.clear()
+    } else {
+      rightMatched = new BitSet(rightMatches.size)
+    }
+  }
+
+  /**
+   * Scan the left and right buffers for the next valid match.
+   *
+   * Note: this method mutates `joinedRow` to point to the latest matching rows in the buffers.
+   * If a left row has no valid matches on the right, or a right row has no valid matches on the
+   * left, then the row is joined with the null row and the result is considered a valid match.
+   *
+   * @return true if a valid match is found, false otherwise.
+   */
+  private def scanNextInBuffered(): Boolean = {
+    while (leftIndex < leftMatches.size) {
+      while (rightIndex < rightMatches.size) {
+        joinedRow(leftMatches(leftIndex), rightMatches(rightIndex))
+        if (boundCondition(joinedRow)) {
+          leftMatched.set(leftIndex)
+          rightMatched.set(rightIndex)
+          rightIndex += 1
+          return true
+        }
+        rightIndex += 1
+      }
+      rightIndex = 0
+      if (!leftMatched.get(leftIndex)) {
+        // the left row has never matched any right row, join it with null row
+        joinedRow(leftMatches(leftIndex), rightNullRow)
+        leftIndex += 1
+        return true
+      }
+      leftIndex += 1
+    }
+
+    while (rightIndex < rightMatches.size) {
+      if (!rightMatched.get(rightIndex)) {
+        // the right row has never matched any left row, join it with null row
+        joinedRow(leftNullRow, rightMatches(rightIndex))
+        rightIndex += 1
+        return true
+      }
+      rightIndex += 1
+    }
+
+    // There are no more valid matches in the left and right buffers
+    false
+  }
+
+  // --- Public methods --------------------------------------------------------------------------
+
+  def getJoinedRow(): JoinedRow = joinedRow
+
+  def advanceNext(): Boolean = {
+    // If we already buffered some matching rows, use them directly
+    if (leftIndex <= leftMatches.size || rightIndex <= rightMatches.size) {
+      if (scanNextInBuffered()) {
+        return true
+      }
+    }
+
+    if (leftRow != null && (leftRowKey.anyNull || rightRow == null)) {
+      joinedRow(leftRow.copy(), rightNullRow)
+      advancedLeft()
+      true
+    } else if (rightRow != null && (rightRowKey.anyNull || leftRow == null)) {
+      joinedRow(leftNullRow, rightRow.copy())
+      advancedRight()
+      true
+    } else if (leftRow != null && rightRow != null) {
+      // Both rows are present and neither have null values,
+      // so we populate the buffers with rows matching the next key
+      val comp = keyOrdering.compare(leftRowKey, rightRowKey)
+      if (comp <= 0) {
+        findMatchingRows(leftRowKey.copy())
+      } else {
+        findMatchingRows(rightRowKey.copy())
+      }
+      scanNextInBuffered()
+      true
+    } else {
+      // Both iterators have been consumed
+      false
+    }
+  }
+}
+
+private class FullOuterIterator(
+    smjScanner: SortMergeFullOuterJoinScanner,
+    resultProj: InternalRow => InternalRow,
+    numRows: LongSQLMetric) extends RowIterator {
+  private[this] val joinedRow: JoinedRow = smjScanner.getJoinedRow()
+
+  override def advanceNext(): Boolean = {
+    val r = smjScanner.advanceNext()
+    if (r) numRows += 1
+    r
+  }
+
+  override def getRow: InternalRow = resultProj(joinedRow)
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala
----------------------------------------------------------------------
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 c9a1459..b71f333 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)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[2/7] spark git commit: [SPARK-14855][SQL] Add "Exec" suffix to physical operators

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala
----------------------------------------------------------------------
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 7c8bc7f..56a3906 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)

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala
new file mode 100644
index 0000000..061d7c7
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala
@@ -0,0 +1,149 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.python
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import net.razorvine.pickle.{Pickler, Unpickler}
+
+import org.apache.spark.TaskContext
+import org.apache.spark.api.python.{ChainedPythonFunctions, PythonRunner}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+
+
+/**
+ * A physical plan that evaluates a [[PythonUDF]], one partition of tuples at a time.
+ *
+ * Python evaluation works by sending the necessary (projected) input data via a socket to an
+ * external Python process, and combine the result from the Python process with the original row.
+ *
+ * For each row we send to Python, we also put it in a queue. For each output row from Python,
+ * 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 BatchEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], child: SparkPlan)
+  extends SparkPlan {
+
+  def children: Seq[SparkPlan] = child :: Nil
+
+  private def collectFunctions(udf: PythonUDF): (ChainedPythonFunctions, Seq[Expression]) = {
+    udf.children match {
+      case Seq(u: PythonUDF) =>
+        val (chained, children) = collectFunctions(u)
+        (ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), children)
+      case children =>
+        // There should not be any other UDFs, or the children can't be evaluated directly.
+        assert(children.forall(_.find(_.isInstanceOf[PythonUDF]).isEmpty))
+        (ChainedPythonFunctions(Seq(udf.func)), udf.children)
+    }
+  }
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    val inputRDD = child.execute().map(_.copy())
+    val bufferSize = inputRDD.conf.getInt("spark.buffer.size", 65536)
+    val reuseWorker = inputRDD.conf.getBoolean("spark.python.worker.reuse", defaultValue = true)
+
+    inputRDD.mapPartitions { iter =>
+      EvaluatePython.registerPicklers()  // register pickler for Row
+
+      // The queue used to buffer input rows so we can drain it to
+      // combine input with output from Python.
+      val queue = new java.util.concurrent.ConcurrentLinkedQueue[InternalRow]()
+
+      val (pyFuncs, inputs) = udfs.map(collectFunctions).unzip
+
+      // flatten all the arguments
+      val allInputs = new ArrayBuffer[Expression]
+      val dataTypes = new ArrayBuffer[DataType]
+      val argOffsets = inputs.map { input =>
+        input.map { e =>
+          if (allInputs.exists(_.semanticEquals(e))) {
+            allInputs.indexWhere(_.semanticEquals(e))
+          } else {
+            allInputs += e
+            dataTypes += e.dataType
+            allInputs.length - 1
+          }
+        }.toArray
+      }.toArray
+      val projection = newMutableProjection(allInputs, child.output)
+      val schema = StructType(dataTypes.map(dt => StructField("", dt)))
+      val needConversion = dataTypes.exists(EvaluatePython.needConversionInPython)
+
+      // enable memo iff we serialize the row with schema (schema and class should be memorized)
+      val pickle = new Pickler(needConversion)
+      // Input iterator to Python: input rows are grouped so we send them in batches to Python.
+      // For each row, add it to the queue.
+      val inputIterator = iter.grouped(100).map { inputRows =>
+        val toBePickled = inputRows.map { inputRow =>
+          queue.add(inputRow)
+          val row = projection(inputRow)
+          if (needConversion) {
+            EvaluatePython.toJava(row, schema)
+          } else {
+            // fast path for these types that does not need conversion in Python
+            val fields = new Array[Any](row.numFields)
+            var i = 0
+            while (i < row.numFields) {
+              val dt = dataTypes(i)
+              fields(i) = EvaluatePython.toJava(row.get(i, dt), dt)
+              i += 1
+            }
+            fields
+          }
+        }.toArray
+        pickle.dumps(toBePickled)
+      }
+
+      val context = TaskContext.get()
+
+      // Output iterator for results from Python.
+      val outputIterator = new PythonRunner(pyFuncs, bufferSize, reuseWorker, true, argOffsets)
+        .compute(inputIterator, context.partitionId(), context)
+
+      val unpickle = new Unpickler
+      val mutableRow = new GenericMutableRow(1)
+      val joined = new JoinedRow
+      val resultType = if (udfs.length == 1) {
+        udfs.head.dataType
+      } else {
+        StructType(udfs.map(u => StructField("", u.dataType, u.nullable)))
+      }
+      val resultProj = UnsafeProjection.create(output, output)
+
+      outputIterator.flatMap { pickedResult =>
+        val unpickledBatch = unpickle.loads(pickedResult)
+        unpickledBatch.asInstanceOf[java.util.ArrayList[Any]].asScala
+      }.map { result =>
+        val row = if (udfs.length == 1) {
+          // fast path for single UDF
+          mutableRow(0) = EvaluatePython.fromJava(result, resultType)
+          mutableRow
+        } else {
+          EvaluatePython.fromJava(result, resultType).asInstanceOf[InternalRow]
+        }
+        resultProj(joined(queue.poll(), row))
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala
----------------------------------------------------------------------
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/BatchPythonEvaluation.scala
deleted file mode 100644
index c49f173..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*    http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.spark.sql.execution.python
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-import net.razorvine.pickle.{Pickler, Unpickler}
-
-import org.apache.spark.TaskContext
-import org.apache.spark.api.python.{ChainedPythonFunctions, PythonRunner}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.types.{DataType, StructField, StructType}
-
-
-/**
- * A physical plan that evaluates a [[PythonUDF]], one partition of tuples at a time.
- *
- * Python evaluation works by sending the necessary (projected) input data via a socket to an
- * external Python process, and combine the result from the Python process with the original row.
- *
- * For each row we send to Python, we also put it in a queue. For each output row from Python,
- * 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)
-  extends SparkPlan {
-
-  def children: Seq[SparkPlan] = child :: Nil
-
-  private def collectFunctions(udf: PythonUDF): (ChainedPythonFunctions, Seq[Expression]) = {
-    udf.children match {
-      case Seq(u: PythonUDF) =>
-        val (chained, children) = collectFunctions(u)
-        (ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), children)
-      case children =>
-        // There should not be any other UDFs, or the children can't be evaluated directly.
-        assert(children.forall(_.find(_.isInstanceOf[PythonUDF]).isEmpty))
-        (ChainedPythonFunctions(Seq(udf.func)), udf.children)
-    }
-  }
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    val inputRDD = child.execute().map(_.copy())
-    val bufferSize = inputRDD.conf.getInt("spark.buffer.size", 65536)
-    val reuseWorker = inputRDD.conf.getBoolean("spark.python.worker.reuse", defaultValue = true)
-
-    inputRDD.mapPartitions { iter =>
-      EvaluatePython.registerPicklers()  // register pickler for Row
-
-      // The queue used to buffer input rows so we can drain it to
-      // combine input with output from Python.
-      val queue = new java.util.concurrent.ConcurrentLinkedQueue[InternalRow]()
-
-      val (pyFuncs, inputs) = udfs.map(collectFunctions).unzip
-
-      // flatten all the arguments
-      val allInputs = new ArrayBuffer[Expression]
-      val dataTypes = new ArrayBuffer[DataType]
-      val argOffsets = inputs.map { input =>
-        input.map { e =>
-          if (allInputs.exists(_.semanticEquals(e))) {
-            allInputs.indexWhere(_.semanticEquals(e))
-          } else {
-            allInputs += e
-            dataTypes += e.dataType
-            allInputs.length - 1
-          }
-        }.toArray
-      }.toArray
-      val projection = newMutableProjection(allInputs, child.output)
-      val schema = StructType(dataTypes.map(dt => StructField("", dt)))
-      val needConversion = dataTypes.exists(EvaluatePython.needConversionInPython)
-
-      // enable memo iff we serialize the row with schema (schema and class should be memorized)
-      val pickle = new Pickler(needConversion)
-      // Input iterator to Python: input rows are grouped so we send them in batches to Python.
-      // For each row, add it to the queue.
-      val inputIterator = iter.grouped(100).map { inputRows =>
-        val toBePickled = inputRows.map { inputRow =>
-          queue.add(inputRow)
-          val row = projection(inputRow)
-          if (needConversion) {
-            EvaluatePython.toJava(row, schema)
-          } else {
-            // fast path for these types that does not need conversion in Python
-            val fields = new Array[Any](row.numFields)
-            var i = 0
-            while (i < row.numFields) {
-              val dt = dataTypes(i)
-              fields(i) = EvaluatePython.toJava(row.get(i, dt), dt)
-              i += 1
-            }
-            fields
-          }
-        }.toArray
-        pickle.dumps(toBePickled)
-      }
-
-      val context = TaskContext.get()
-
-      // Output iterator for results from Python.
-      val outputIterator = new PythonRunner(pyFuncs, bufferSize, reuseWorker, true, argOffsets)
-        .compute(inputIterator, context.partitionId(), context)
-
-      val unpickle = new Unpickler
-      val mutableRow = new GenericMutableRow(1)
-      val joined = new JoinedRow
-      val resultType = if (udfs.length == 1) {
-        udfs.head.dataType
-      } else {
-        StructType(udfs.map(u => StructField("", u.dataType, u.nullable)))
-      }
-      val resultProj = UnsafeProjection.create(output, output)
-
-      outputIterator.flatMap { pickedResult =>
-        val unpickledBatch = unpickle.loads(pickedResult)
-        unpickledBatch.asInstanceOf[java.util.ArrayList[Any]].asScala
-      }.map { result =>
-        val row = if (udfs.length == 1) {
-          // fast path for single UDF
-          mutableRow(0) = EvaluatePython.fromJava(result, resultType)
-          mutableRow
-        } else {
-          EvaluatePython.fromJava(result, resultType).asInstanceOf[InternalRow]
-        }
-        resultProj(joined(queue.poll(), row))
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala
----------------------------------------------------------------------
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 d72b3d3..ab19236 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
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
----------------------------------------------------------------------
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 81244ed..a1a1108 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))

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala
----------------------------------------------------------------------
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 5957747..de4305f 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(

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
----------------------------------------------------------------------
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 c023cc5..1341e45 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala
----------------------------------------------------------------------
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 c6fcb69..1959f1e 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
----------------------------------------------------------------------
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 82b79c7..4aea21e 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)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
----------------------------------------------------------------------
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 351b03b..19fe29a 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)
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
----------------------------------------------------------------------
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 067a62d..0414fa1 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
----------------------------------------------------------------------
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 e953a6e..4c18784 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)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala
----------------------------------------------------------------------
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 a87a41c..9e5a41d 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(

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
----------------------------------------------------------------------
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 9e64049..84f0c00 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)

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala
----------------------------------------------------------------------
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 17f2343..ba16810 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)

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
----------------------------------------------------------------------
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 bdbcf84..3b2911d 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) {

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala
----------------------------------------------------------------------
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 2963a85..a19ea51 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala
----------------------------------------------------------------------
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 7784776..ebeb39b 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
       )

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala
----------------------------------------------------------------------
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 a4c6d07..fba04d0 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)
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala
----------------------------------------------------------------------
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 d7cf1dc..233104a 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)))
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala
----------------------------------------------------------------------
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 4f185ed..9164074 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")

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala
----------------------------------------------------------------------
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 50cd03a..fb70dbd 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}")

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala
----------------------------------------------------------------------
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 babe7ef..b9df43d 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")
   }
 
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala
----------------------------------------------------------------------
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 8cdfa8a..bc838ee 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)
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala
----------------------------------------------------------------------
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 3cb3ef1..933f32e 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)
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala
----------------------------------------------------------------------
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 4cacb20..c26cb84 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)
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
----------------------------------------------------------------------
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 f66deea..c24abf1 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)

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala
----------------------------------------------------------------------
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 19e34b4..1470777 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")


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[6/7] spark git commit: [SPARK-14855][SQL] Add "Exec" suffix to physical operators

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala
----------------------------------------------------------------------
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/WholeStageCodegen.scala
deleted file mode 100644
index 944962b..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala
+++ /dev/null
@@ -1,492 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution
-
-import org.apache.spark.{broadcast, TaskContext}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen._
-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.metric.{LongSQLMetricValue, SQLMetrics}
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types._
-
-/**
- * An interface for those physical operators that support codegen.
- */
-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 _ => nodeName.toLowerCase
-  }
-
-  /**
-   * Creates a metric using the specified name.
-   *
-   * @return name of the variable representing the metric
-   */
-  def metricTerm(ctx: CodegenContext, name: String): String = {
-    val metric = ctx.addReferenceObj(name, longMetric(name))
-    val value = ctx.freshName("metricValue")
-    val cls = classOf[LongSQLMetricValue].getName
-    ctx.addMutableState(cls, value, s"$value = ($cls) $metric.localValue();")
-    value
-  }
-
-  /**
-   * Whether this SparkPlan support whole stage codegen or not.
-   */
-  def supportCodegen: Boolean = true
-
-  /**
-   * Which SparkPlan is calling produce() of this one. It's itself for the first SparkPlan.
-   */
-  protected var parent: CodegenSupport = null
-
-  /**
-   * Returns all the RDDs of InternalRow which generates the input rows.
-   *
-   * Note: right now we support up to two RDDs.
-   */
-  def inputRDDs(): Seq[RDD[InternalRow]]
-
-  /**
-   * Returns Java source code to process the rows from input RDD.
-   */
-  final def produce(ctx: CodegenContext, parent: CodegenSupport): String = executeQuery {
-    this.parent = parent
-    ctx.freshNamePrefix = variablePrefix
-    s"""
-       |/*** PRODUCE: ${toCommentSafeString(this.simpleString)} */
-       |${doProduce(ctx)}
-     """.stripMargin
-  }
-
-  /**
-   * Generate the Java source code to process, should be overridden by subclass to support codegen.
-   *
-   * doProduce() usually generate the framework, for example, aggregation could generate this:
-   *
-   *   if (!initialized) {
-   *     # create a hash map, then build the aggregation hash map
-   *     # call child.produce()
-   *     initialized = true;
-   *   }
-   *   while (hashmap.hasNext()) {
-   *     row = hashmap.next();
-   *     # build the aggregation results
-   *     # create variables for results
-   *     # call consume(), which will call parent.doConsume()
-   *      if (shouldStop()) return;
-   *   }
-   */
-  protected def doProduce(ctx: CodegenContext): String
-
-  /**
-   * Consume the generated columns or row from current SparkPlan, call it's parent's doConsume().
-   */
-  final def consume(ctx: CodegenContext, outputVars: Seq[ExprCode], row: String = null): String = {
-    val inputVars =
-      if (row != null) {
-        ctx.currentVars = null
-        ctx.INPUT_ROW = row
-        output.zipWithIndex.map { case (attr, i) =>
-          BoundReference(i, attr.dataType, attr.nullable).genCode(ctx)
-        }
-      } else {
-        assert(outputVars != null)
-        assert(outputVars.length == output.length)
-        // outputVars will be used to generate the code for UnsafeRow, so we should copy them
-        outputVars.map(_.copy())
-      }
-
-    val rowVar = if (row != null) {
-      ExprCode("", "false", row)
-    } else {
-      if (outputVars.nonEmpty) {
-        val colExprs = output.zipWithIndex.map { case (attr, i) =>
-          BoundReference(i, attr.dataType, attr.nullable)
-        }
-        val evaluateInputs = evaluateVariables(outputVars)
-        // generate the code to create a UnsafeRow
-        ctx.currentVars = outputVars
-        val ev = GenerateUnsafeProjection.createCode(ctx, colExprs, false)
-        val code = s"""
-          |$evaluateInputs
-          |${ev.code.trim}
-         """.stripMargin.trim
-        ExprCode(code, "false", ev.value)
-      } else {
-        // There is no columns
-        ExprCode("", "false", "unsafeRow")
-      }
-    }
-
-    ctx.freshNamePrefix = parent.variablePrefix
-    val evaluated = evaluateRequiredVariables(output, inputVars, parent.usedInputs)
-    s"""
-       |
-       |/*** CONSUME: ${toCommentSafeString(parent.simpleString)} */
-       |$evaluated
-       |${parent.doConsume(ctx, inputVars, rowVar)}
-     """.stripMargin
-  }
-
-  /**
-   * Returns source code to evaluate all the variables, and clear the code of them, to prevent
-   * them to be evaluated twice.
-   */
-  protected def evaluateVariables(variables: Seq[ExprCode]): String = {
-    val evaluate = variables.filter(_.code != "").map(_.code.trim).mkString("\n")
-    variables.foreach(_.code = "")
-    evaluate
-  }
-
-  /**
-   * Returns source code to evaluate the variables for required attributes, and clear the code
-   * of evaluated variables, to prevent them to be evaluated twice.
-   */
-  protected def evaluateRequiredVariables(
-      attributes: Seq[Attribute],
-      variables: Seq[ExprCode],
-      required: AttributeSet): String = {
-    val evaluateVars = new StringBuilder
-    variables.zipWithIndex.foreach { case (ev, i) =>
-      if (ev.code != "" && required.contains(attributes(i))) {
-        evaluateVars.append(ev.code.trim + "\n")
-        ev.code = ""
-      }
-    }
-    evaluateVars.toString()
-  }
-
-  /**
-   * The subset of inputSet those should be evaluated before this plan.
-   *
-   * We will use this to insert some code to access those columns that are actually used by current
-   * plan before calling doConsume().
-   */
-  def usedInputs: AttributeSet = references
-
-  /**
-   * Generate the Java source code to process the rows from child SparkPlan.
-   *
-   * This should be override by subclass to support codegen.
-   *
-   * For example, Filter will generate the code like this:
-   *
-   *   # code to evaluate the predicate expression, result is isNull1 and value2
-   *   if (isNull1 || !value2) continue;
-   *   # call consume(), which will call parent.doConsume()
-   *
-   * Note: A plan can either consume the rows as UnsafeRow (row), or a list of variables (input).
-   */
-  def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
-    throw new UnsupportedOperationException
-  }
-}
-
-
-/**
- * InputAdapter is used to hide a SparkPlan from a subtree that support codegen.
- *
- * 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 {
-
-  override def output: Seq[Attribute] = child.output
-  override def outputPartitioning: Partitioning = child.outputPartitioning
-  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
-
-  override def doExecute(): RDD[InternalRow] = {
-    child.execute()
-  }
-
-  override def doExecuteBroadcast[T](): broadcast.Broadcast[T] = {
-    child.doExecuteBroadcast()
-  }
-
-  override def inputRDDs(): Seq[RDD[InternalRow]] = {
-    child.execute() :: Nil
-  }
-
-  override def doProduce(ctx: CodegenContext): String = {
-    val input = ctx.freshName("input")
-    // Right now, InputAdapter is only used when there is one input RDD.
-    ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];")
-    val row = ctx.freshName("row")
-    s"""
-       | while ($input.hasNext()) {
-       |   InternalRow $row = (InternalRow) $input.next();
-       |   ${consume(ctx, null, row).trim}
-       |   if (shouldStop()) return;
-       | }
-     """.stripMargin
-  }
-
-  override def simpleString: String = "INPUT"
-
-  override def treeChildren: Seq[SparkPlan] = Nil
-}
-
-object WholeStageCodegen {
-  val PIPELINE_DURATION_METRIC = "duration"
-}
-
-/**
- * WholeStageCodegen compile a subtree of plans that support codegen together into single Java
- * function.
- *
- * Here is the call graph of to generate Java source (plan A support codegen, but plan B does not):
- *
- *   WholeStageCodegen       Plan A               FakeInput        Plan B
- * =========================================================================
- *
- * -> execute()
- *     |
- *  doExecute() --------->   inputRDDs() -------> inputRDDs() ------> execute()
- *     |
- *     +----------------->   produce()
- *                             |
- *                          doProduce()  -------> produce()
- *                                                   |
- *                                                doProduce()
- *                                                   |
- *                         doConsume() <--------- consume()
- *                             |
- *  doConsume()  <--------  consume()
- *
- * SparkPlan A should override doProduce() and doConsume().
- *
- * 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 {
-
-  override def output: Seq[Attribute] = child.output
-  override def outputPartitioning: Partitioning = child.outputPartitioning
-  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
-
-  override private[sql] lazy val metrics = Map(
-    "pipelineTime" -> SQLMetrics.createTimingMetric(sparkContext,
-      WholeStageCodegen.PIPELINE_DURATION_METRIC))
-
-  /**
-   * Generates code for this subtree.
-   *
-   * @return the tuple of the codegen context and the actual generated source.
-   */
-  def doCodeGen(): (CodegenContext, String) = {
-    val ctx = new CodegenContext
-    val code = child.asInstanceOf[CodegenSupport].produce(ctx, this)
-    val source = s"""
-      public Object generate(Object[] references) {
-        return new GeneratedIterator(references);
-      }
-
-      /** Codegened pipeline for:
-       * ${toCommentSafeString(child.treeString.trim)}
-       */
-      final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
-
-        private Object[] references;
-        ${ctx.declareMutableStates()}
-
-        public GeneratedIterator(Object[] references) {
-          this.references = references;
-        }
-
-        public void init(int index, scala.collection.Iterator inputs[]) {
-          partitionIndex = index;
-          ${ctx.initMutableStates()}
-        }
-
-        ${ctx.declareAddedFunctions()}
-
-        protected void processNext() throws java.io.IOException {
-          ${code.trim}
-        }
-      }
-      """.trim
-
-    // try to compile, helpful for debug
-    val cleanedSource = CodeFormatter.stripExtraNewLines(source)
-    logDebug(s"\n${CodeFormatter.format(cleanedSource)}")
-    CodeGenerator.compile(cleanedSource)
-    (ctx, cleanedSource)
-  }
-
-  override def doExecute(): RDD[InternalRow] = {
-    val (ctx, cleanedSource) = doCodeGen()
-    val references = ctx.references.toArray
-
-    val durationMs = longMetric("pipelineTime")
-
-    val rdds = child.asInstanceOf[CodegenSupport].inputRDDs()
-    assert(rdds.size <= 2, "Up to two input RDDs can be supported")
-    if (rdds.length == 1) {
-      rdds.head.mapPartitionsWithIndex { (index, iter) =>
-        val clazz = CodeGenerator.compile(cleanedSource)
-        val buffer = clazz.generate(references).asInstanceOf[BufferedRowIterator]
-        buffer.init(index, Array(iter))
-        new Iterator[InternalRow] {
-          override def hasNext: Boolean = {
-            val v = buffer.hasNext
-            if (!v) durationMs += buffer.durationMs()
-            v
-          }
-          override def next: InternalRow = buffer.next()
-        }
-      }
-    } else {
-      // Right now, we support up to two input RDDs.
-      rdds.head.zipPartitions(rdds(1)) { (leftIter, rightIter) =>
-        val partitionIndex = TaskContext.getPartitionId()
-        val clazz = CodeGenerator.compile(cleanedSource)
-        val buffer = clazz.generate(references).asInstanceOf[BufferedRowIterator]
-        buffer.init(partitionIndex, Array(leftIter, rightIter))
-        new Iterator[InternalRow] {
-          override def hasNext: Boolean = {
-            val v = buffer.hasNext
-            if (!v) durationMs += buffer.durationMs()
-            v
-          }
-          override def next: InternalRow = buffer.next()
-        }
-      }
-    }
-  }
-
-  override def inputRDDs(): Seq[RDD[InternalRow]] = {
-    throw new UnsupportedOperationException
-  }
-
-  override def doProduce(ctx: CodegenContext): String = {
-    throw new UnsupportedOperationException
-  }
-
-  override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
-    val doCopy = if (ctx.copyResult) {
-      ".copy()"
-    } else {
-      ""
-    }
-    s"""
-      |${row.code}
-      |append(${row.value}$doCopy);
-     """.stripMargin.trim
-  }
-
-  override def innerChildren: Seq[SparkPlan] = {
-    child :: Nil
-  }
-
-  private def collectInputs(plan: SparkPlan): Seq[SparkPlan] = plan match {
-    case InputAdapter(c) => c :: Nil
-    case other => other.children.flatMap(collectInputs)
-  }
-
-  override def treeChildren: Seq[SparkPlan] = {
-    collectInputs(child)
-  }
-
-  override def simpleString: String = "WholeStageCodegen"
-}
-
-
-/**
- * Find the chained plans that support codegen, collapse them together as WholeStageCodegen.
- */
-case class CollapseCodegenStages(conf: SQLConf) extends Rule[SparkPlan] {
-
-  private def supportCodegen(e: Expression): Boolean = e match {
-    case e: LeafExpression => true
-    // CodegenFallback requires the input to be an InternalRow
-    case e: CodegenFallback => false
-    case _ => true
-  }
-
-  private def numOfNestedFields(dataType: DataType): Int = dataType match {
-    case dt: StructType => dt.fields.map(f => numOfNestedFields(f.dataType)).sum
-    case m: MapType => numOfNestedFields(m.keyType) + numOfNestedFields(m.valueType)
-    case a: ArrayType => numOfNestedFields(a.elementType)
-    case u: UserDefinedType[_] => numOfNestedFields(u.sqlType)
-    case _ => 1
-  }
-
-  private def supportCodegen(plan: SparkPlan): Boolean = plan match {
-    case plan: CodegenSupport if plan.supportCodegen =>
-      val willFallback = plan.expressions.exists(_.find(e => !supportCodegen(e)).isDefined)
-      // the generated code will be huge if there are too many columns
-      val hasTooManyOutputFields =
-        numOfNestedFields(plan.schema) > conf.wholeStageMaxNumFields
-      val hasTooManyInputFields =
-        plan.children.map(p => numOfNestedFields(p.schema)).exists(_ > conf.wholeStageMaxNumFields)
-      !willFallback && !hasTooManyOutputFields && !hasTooManyInputFields
-    case _ => false
-  }
-
-  /**
-   * 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 =>
-      // The children of SortMergeJoin should do codegen separately.
-      j.copy(left = InputAdapter(insertWholeStageCodegen(left)),
-        right = InputAdapter(insertWholeStageCodegen(right)))
-    case p if !supportCodegen(p) =>
-      // collapse them recursively
-      InputAdapter(insertWholeStageCodegen(p))
-    case p =>
-      p.withNewChildren(p.children.map(insertInputAdapter))
-  }
-
-  /**
-   * Inserts a WholeStageCodegen on top of those that support codegen.
-   */
-  private def insertWholeStageCodegen(plan: SparkPlan): SparkPlan = plan match {
-    // For operators that will output domain object, do not insert WholeStageCodegen for it as
-    // domain object can not be written into unsafe row.
-    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))
-    case other =>
-      other.withNewChildren(other.children.map(insertWholeStageCodegen))
-  }
-
-  def apply(plan: SparkPlan): SparkPlan = {
-    if (conf.wholeStageEnabled) {
-      insertWholeStageCodegen(plan)
-    } else {
-      plan
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
new file mode 100644
index 0000000..6a03bd0
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
@@ -0,0 +1,492 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import org.apache.spark.{broadcast, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen._
+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.{BroadcastHashJoinExec, SortMergeJoinExec}
+import org.apache.spark.sql.execution.metric.{LongSQLMetricValue, SQLMetrics}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * An interface for those physical operators that support codegen.
+ */
+trait CodegenSupport extends SparkPlan {
+
+  /** Prefix used in the current operator's variable names. */
+  private def variablePrefix: String = this match {
+    case _: TungstenAggregate => "agg"
+    case _: BroadcastHashJoinExec => "bhj"
+    case _: SortMergeJoinExec => "smj"
+    case _: RDDScanExec => "rdd"
+    case _: DataSourceScanExec => "scan"
+    case _ => nodeName.toLowerCase
+  }
+
+  /**
+   * Creates a metric using the specified name.
+   *
+   * @return name of the variable representing the metric
+   */
+  def metricTerm(ctx: CodegenContext, name: String): String = {
+    val metric = ctx.addReferenceObj(name, longMetric(name))
+    val value = ctx.freshName("metricValue")
+    val cls = classOf[LongSQLMetricValue].getName
+    ctx.addMutableState(cls, value, s"$value = ($cls) $metric.localValue();")
+    value
+  }
+
+  /**
+   * Whether this SparkPlan support whole stage codegen or not.
+   */
+  def supportCodegen: Boolean = true
+
+  /**
+   * Which SparkPlan is calling produce() of this one. It's itself for the first SparkPlan.
+   */
+  protected var parent: CodegenSupport = null
+
+  /**
+   * Returns all the RDDs of InternalRow which generates the input rows.
+   *
+   * Note: right now we support up to two RDDs.
+   */
+  def inputRDDs(): Seq[RDD[InternalRow]]
+
+  /**
+   * Returns Java source code to process the rows from input RDD.
+   */
+  final def produce(ctx: CodegenContext, parent: CodegenSupport): String = executeQuery {
+    this.parent = parent
+    ctx.freshNamePrefix = variablePrefix
+    s"""
+       |/*** PRODUCE: ${toCommentSafeString(this.simpleString)} */
+       |${doProduce(ctx)}
+     """.stripMargin
+  }
+
+  /**
+   * Generate the Java source code to process, should be overridden by subclass to support codegen.
+   *
+   * doProduce() usually generate the framework, for example, aggregation could generate this:
+   *
+   *   if (!initialized) {
+   *     # create a hash map, then build the aggregation hash map
+   *     # call child.produce()
+   *     initialized = true;
+   *   }
+   *   while (hashmap.hasNext()) {
+   *     row = hashmap.next();
+   *     # build the aggregation results
+   *     # create variables for results
+   *     # call consume(), which will call parent.doConsume()
+   *      if (shouldStop()) return;
+   *   }
+   */
+  protected def doProduce(ctx: CodegenContext): String
+
+  /**
+   * Consume the generated columns or row from current SparkPlan, call it's parent's doConsume().
+   */
+  final def consume(ctx: CodegenContext, outputVars: Seq[ExprCode], row: String = null): String = {
+    val inputVars =
+      if (row != null) {
+        ctx.currentVars = null
+        ctx.INPUT_ROW = row
+        output.zipWithIndex.map { case (attr, i) =>
+          BoundReference(i, attr.dataType, attr.nullable).genCode(ctx)
+        }
+      } else {
+        assert(outputVars != null)
+        assert(outputVars.length == output.length)
+        // outputVars will be used to generate the code for UnsafeRow, so we should copy them
+        outputVars.map(_.copy())
+      }
+
+    val rowVar = if (row != null) {
+      ExprCode("", "false", row)
+    } else {
+      if (outputVars.nonEmpty) {
+        val colExprs = output.zipWithIndex.map { case (attr, i) =>
+          BoundReference(i, attr.dataType, attr.nullable)
+        }
+        val evaluateInputs = evaluateVariables(outputVars)
+        // generate the code to create a UnsafeRow
+        ctx.currentVars = outputVars
+        val ev = GenerateUnsafeProjection.createCode(ctx, colExprs, false)
+        val code = s"""
+          |$evaluateInputs
+          |${ev.code.trim}
+         """.stripMargin.trim
+        ExprCode(code, "false", ev.value)
+      } else {
+        // There is no columns
+        ExprCode("", "false", "unsafeRow")
+      }
+    }
+
+    ctx.freshNamePrefix = parent.variablePrefix
+    val evaluated = evaluateRequiredVariables(output, inputVars, parent.usedInputs)
+    s"""
+       |
+       |/*** CONSUME: ${toCommentSafeString(parent.simpleString)} */
+       |$evaluated
+       |${parent.doConsume(ctx, inputVars, rowVar)}
+     """.stripMargin
+  }
+
+  /**
+   * Returns source code to evaluate all the variables, and clear the code of them, to prevent
+   * them to be evaluated twice.
+   */
+  protected def evaluateVariables(variables: Seq[ExprCode]): String = {
+    val evaluate = variables.filter(_.code != "").map(_.code.trim).mkString("\n")
+    variables.foreach(_.code = "")
+    evaluate
+  }
+
+  /**
+   * Returns source code to evaluate the variables for required attributes, and clear the code
+   * of evaluated variables, to prevent them to be evaluated twice.
+   */
+  protected def evaluateRequiredVariables(
+      attributes: Seq[Attribute],
+      variables: Seq[ExprCode],
+      required: AttributeSet): String = {
+    val evaluateVars = new StringBuilder
+    variables.zipWithIndex.foreach { case (ev, i) =>
+      if (ev.code != "" && required.contains(attributes(i))) {
+        evaluateVars.append(ev.code.trim + "\n")
+        ev.code = ""
+      }
+    }
+    evaluateVars.toString()
+  }
+
+  /**
+   * The subset of inputSet those should be evaluated before this plan.
+   *
+   * We will use this to insert some code to access those columns that are actually used by current
+   * plan before calling doConsume().
+   */
+  def usedInputs: AttributeSet = references
+
+  /**
+   * Generate the Java source code to process the rows from child SparkPlan.
+   *
+   * This should be override by subclass to support codegen.
+   *
+   * For example, Filter will generate the code like this:
+   *
+   *   # code to evaluate the predicate expression, result is isNull1 and value2
+   *   if (isNull1 || !value2) continue;
+   *   # call consume(), which will call parent.doConsume()
+   *
+   * Note: A plan can either consume the rows as UnsafeRow (row), or a list of variables (input).
+   */
+  def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
+    throw new UnsupportedOperationException
+  }
+}
+
+
+/**
+ * InputAdapter is used to hide a SparkPlan from a subtree that support codegen.
+ *
+ * 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 UnaryExecNode with CodegenSupport {
+
+  override def output: Seq[Attribute] = child.output
+  override def outputPartitioning: Partitioning = child.outputPartitioning
+  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
+
+  override def doExecute(): RDD[InternalRow] = {
+    child.execute()
+  }
+
+  override def doExecuteBroadcast[T](): broadcast.Broadcast[T] = {
+    child.doExecuteBroadcast()
+  }
+
+  override def inputRDDs(): Seq[RDD[InternalRow]] = {
+    child.execute() :: Nil
+  }
+
+  override def doProduce(ctx: CodegenContext): String = {
+    val input = ctx.freshName("input")
+    // Right now, InputAdapter is only used when there is one input RDD.
+    ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];")
+    val row = ctx.freshName("row")
+    s"""
+       | while ($input.hasNext()) {
+       |   InternalRow $row = (InternalRow) $input.next();
+       |   ${consume(ctx, null, row).trim}
+       |   if (shouldStop()) return;
+       | }
+     """.stripMargin
+  }
+
+  override def simpleString: String = "INPUT"
+
+  override def treeChildren: Seq[SparkPlan] = Nil
+}
+
+object WholeStageCodegenExec {
+  val PIPELINE_DURATION_METRIC = "duration"
+}
+
+/**
+ * WholeStageCodegen compile a subtree of plans that support codegen together into single Java
+ * function.
+ *
+ * Here is the call graph of to generate Java source (plan A support codegen, but plan B does not):
+ *
+ *   WholeStageCodegen       Plan A               FakeInput        Plan B
+ * =========================================================================
+ *
+ * -> execute()
+ *     |
+ *  doExecute() --------->   inputRDDs() -------> inputRDDs() ------> execute()
+ *     |
+ *     +----------------->   produce()
+ *                             |
+ *                          doProduce()  -------> produce()
+ *                                                   |
+ *                                                doProduce()
+ *                                                   |
+ *                         doConsume() <--------- consume()
+ *                             |
+ *  doConsume()  <--------  consume()
+ *
+ * SparkPlan A should override doProduce() and doConsume().
+ *
+ * doCodeGen() will create a CodeGenContext, which will hold a list of variables for input,
+ * used to generated code for BoundReference.
+ */
+case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport {
+
+  override def output: Seq[Attribute] = child.output
+  override def outputPartitioning: Partitioning = child.outputPartitioning
+  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
+
+  override private[sql] lazy val metrics = Map(
+    "pipelineTime" -> SQLMetrics.createTimingMetric(sparkContext,
+      WholeStageCodegenExec.PIPELINE_DURATION_METRIC))
+
+  /**
+   * Generates code for this subtree.
+   *
+   * @return the tuple of the codegen context and the actual generated source.
+   */
+  def doCodeGen(): (CodegenContext, String) = {
+    val ctx = new CodegenContext
+    val code = child.asInstanceOf[CodegenSupport].produce(ctx, this)
+    val source = s"""
+      public Object generate(Object[] references) {
+        return new GeneratedIterator(references);
+      }
+
+      /** Codegened pipeline for:
+       * ${toCommentSafeString(child.treeString.trim)}
+       */
+      final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
+
+        private Object[] references;
+        ${ctx.declareMutableStates()}
+
+        public GeneratedIterator(Object[] references) {
+          this.references = references;
+        }
+
+        public void init(int index, scala.collection.Iterator inputs[]) {
+          partitionIndex = index;
+          ${ctx.initMutableStates()}
+        }
+
+        ${ctx.declareAddedFunctions()}
+
+        protected void processNext() throws java.io.IOException {
+          ${code.trim}
+        }
+      }
+      """.trim
+
+    // try to compile, helpful for debug
+    val cleanedSource = CodeFormatter.stripExtraNewLines(source)
+    logDebug(s"\n${CodeFormatter.format(cleanedSource)}")
+    CodeGenerator.compile(cleanedSource)
+    (ctx, cleanedSource)
+  }
+
+  override def doExecute(): RDD[InternalRow] = {
+    val (ctx, cleanedSource) = doCodeGen()
+    val references = ctx.references.toArray
+
+    val durationMs = longMetric("pipelineTime")
+
+    val rdds = child.asInstanceOf[CodegenSupport].inputRDDs()
+    assert(rdds.size <= 2, "Up to two input RDDs can be supported")
+    if (rdds.length == 1) {
+      rdds.head.mapPartitionsWithIndex { (index, iter) =>
+        val clazz = CodeGenerator.compile(cleanedSource)
+        val buffer = clazz.generate(references).asInstanceOf[BufferedRowIterator]
+        buffer.init(index, Array(iter))
+        new Iterator[InternalRow] {
+          override def hasNext: Boolean = {
+            val v = buffer.hasNext
+            if (!v) durationMs += buffer.durationMs()
+            v
+          }
+          override def next: InternalRow = buffer.next()
+        }
+      }
+    } else {
+      // Right now, we support up to two input RDDs.
+      rdds.head.zipPartitions(rdds(1)) { (leftIter, rightIter) =>
+        val partitionIndex = TaskContext.getPartitionId()
+        val clazz = CodeGenerator.compile(cleanedSource)
+        val buffer = clazz.generate(references).asInstanceOf[BufferedRowIterator]
+        buffer.init(partitionIndex, Array(leftIter, rightIter))
+        new Iterator[InternalRow] {
+          override def hasNext: Boolean = {
+            val v = buffer.hasNext
+            if (!v) durationMs += buffer.durationMs()
+            v
+          }
+          override def next: InternalRow = buffer.next()
+        }
+      }
+    }
+  }
+
+  override def inputRDDs(): Seq[RDD[InternalRow]] = {
+    throw new UnsupportedOperationException
+  }
+
+  override def doProduce(ctx: CodegenContext): String = {
+    throw new UnsupportedOperationException
+  }
+
+  override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
+    val doCopy = if (ctx.copyResult) {
+      ".copy()"
+    } else {
+      ""
+    }
+    s"""
+      |${row.code}
+      |append(${row.value}$doCopy);
+     """.stripMargin.trim
+  }
+
+  override def innerChildren: Seq[SparkPlan] = {
+    child :: Nil
+  }
+
+  private def collectInputs(plan: SparkPlan): Seq[SparkPlan] = plan match {
+    case InputAdapter(c) => c :: Nil
+    case other => other.children.flatMap(collectInputs)
+  }
+
+  override def treeChildren: Seq[SparkPlan] = {
+    collectInputs(child)
+  }
+
+  override def simpleString: String = "WholeStageCodegen"
+}
+
+
+/**
+ * Find the chained plans that support codegen, collapse them together as WholeStageCodegen.
+ */
+case class CollapseCodegenStages(conf: SQLConf) extends Rule[SparkPlan] {
+
+  private def supportCodegen(e: Expression): Boolean = e match {
+    case e: LeafExpression => true
+    // CodegenFallback requires the input to be an InternalRow
+    case e: CodegenFallback => false
+    case _ => true
+  }
+
+  private def numOfNestedFields(dataType: DataType): Int = dataType match {
+    case dt: StructType => dt.fields.map(f => numOfNestedFields(f.dataType)).sum
+    case m: MapType => numOfNestedFields(m.keyType) + numOfNestedFields(m.valueType)
+    case a: ArrayType => numOfNestedFields(a.elementType)
+    case u: UserDefinedType[_] => numOfNestedFields(u.sqlType)
+    case _ => 1
+  }
+
+  private def supportCodegen(plan: SparkPlan): Boolean = plan match {
+    case plan: CodegenSupport if plan.supportCodegen =>
+      val willFallback = plan.expressions.exists(_.find(e => !supportCodegen(e)).isDefined)
+      // the generated code will be huge if there are too many columns
+      val hasTooManyOutputFields =
+        numOfNestedFields(plan.schema) > conf.wholeStageMaxNumFields
+      val hasTooManyInputFields =
+        plan.children.map(p => numOfNestedFields(p.schema)).exists(_ > conf.wholeStageMaxNumFields)
+      !willFallback && !hasTooManyOutputFields && !hasTooManyInputFields
+    case _ => false
+  }
+
+  /**
+   * Inserts a InputAdapter on top of those that do not support codegen.
+   */
+  private def insertInputAdapter(plan: SparkPlan): SparkPlan = plan match {
+    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)))
+    case p if !supportCodegen(p) =>
+      // collapse them recursively
+      InputAdapter(insertWholeStageCodegen(p))
+    case p =>
+      p.withNewChildren(p.children.map(insertInputAdapter))
+  }
+
+  /**
+   * Inserts a WholeStageCodegen on top of those that support codegen.
+   */
+  private def insertWholeStageCodegen(plan: SparkPlan): SparkPlan = plan match {
+    // For operators that will output domain object, do not insert WholeStageCodegen for it as
+    // domain object can not be written into unsafe row.
+    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) =>
+      WholeStageCodegenExec(insertInputAdapter(plan))
+    case other =>
+      other.withNewChildren(other.children.map(insertWholeStageCodegen))
+  }
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.wholeStageEnabled) {
+      insertWholeStageCodegen(plan)
+    } else {
+      plan
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala
----------------------------------------------------------------------
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/Window.scala
deleted file mode 100644
index a46d0e0..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala
+++ /dev/null
@@ -1,1008 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution
-
-import java.util
-
-import scala.collection.mutable
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.spark.{SparkEnv, TaskContext}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate._
-import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.sql.types.IntegerType
-import org.apache.spark.util.collection.unsafe.sort.{UnsafeExternalSorter, UnsafeSorterIterator}
-
-/**
- * This class calculates and outputs (windowed) aggregates over the rows in a single (sorted)
- * partition. The aggregates are calculated for each row in the group. Special processing
- * instructions, frames, are used to calculate these aggregates. Frames are processed in the order
- * specified in the window specification (the ORDER BY ... clause). There are four different frame
- * types:
- * - Entire partition: The frame is the entire partition, i.e.
- *   UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING. For this case, window function will take all
- *   rows as inputs and be evaluated once.
- * - Growing frame: We only add new rows into the frame, i.e. UNBOUNDED PRECEDING AND ....
- *   Every time we move to a new row to process, we add some rows to the frame. We do not remove
- *   rows from this frame.
- * - Shrinking frame: We only remove rows from the frame, i.e. ... AND UNBOUNDED FOLLOWING.
- *   Every time we move to a new row to process, we remove some rows from the frame. We do not add
- *   rows to this frame.
- * - Moving frame: Every time we move to a new row to process, we remove some rows from the frame
- *   and we add some rows to the frame. Examples are:
- *     1 PRECEDING AND CURRENT ROW and 1 FOLLOWING AND 2 FOLLOWING.
- * - Offset frame: The frame consist of one row, which is an offset number of rows away from the
- *   current row. Only [[OffsetWindowFunction]]s can be processed in an offset frame.
- *
- * Different frame boundaries can be used in Growing, Shrinking and Moving frames. A frame
- * boundary can be either Row or Range based:
- * - Row Based: A row based boundary is based on the position of the row within the partition.
- *   An offset indicates the number of rows above or below the current row, the frame for the
- *   current row starts or ends. For instance, given a row based sliding frame with a lower bound
- *   offset of -1 and a upper bound offset of +2. The frame for row with index 5 would range from
- *   index 4 to index 6.
- * - Range based: A range based boundary is based on the actual value of the ORDER BY
- *   expression(s). An offset is used to alter the value of the ORDER BY expression, for
- *   instance if the current order by expression has a value of 10 and the lower bound offset
- *   is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a
- *   number of constraints on the ORDER BY expressions: there can be only one expression and this
- *   expression must have a numerical data type. An exception can be made when the offset is 0,
- *   because no value modification is needed, in this case multiple and non-numeric ORDER BY
- *   expression are allowed.
- *
- * This is quite an expensive operator because every row for a single group must be in the same
- * partition and partitions must be sorted according to the grouping and sort order. The operator
- * requires the planner to take care of the partitioning and sorting.
- *
- * The operator is semi-blocking. The window functions and aggregates are calculated one group at
- * a time, the result will only be made available after the processing for the entire group has
- * finished. The operator is able to process different frame configurations at the same time. This
- * is done by delegating the actual frame processing (i.e. calculation of the window functions) to
- * specialized classes, see [[WindowFunctionFrame]], which take care of their own frame type:
- * Entire Partition, Sliding, Growing & Shrinking. Boundary evaluation is also delegated to a pair
- * of specialized classes: [[RowBoundOrdering]] & [[RangeBoundOrdering]].
- */
-case class Window(
-    windowExpression: Seq[NamedExpression],
-    partitionSpec: Seq[Expression],
-    orderSpec: Seq[SortOrder],
-    child: SparkPlan)
-  extends UnaryNode {
-
-  override def output: Seq[Attribute] =
-    child.output ++ windowExpression.map(_.toAttribute)
-
-  override def requiredChildDistribution: Seq[Distribution] = {
-    if (partitionSpec.isEmpty) {
-      // Only show warning when the number of bytes is larger than 100 MB?
-      logWarning("No Partition Defined for Window operation! Moving all data to a single "
-        + "partition, this can cause serious performance degradation.")
-      AllTuples :: Nil
-    } else ClusteredDistribution(partitionSpec) :: Nil
-  }
-
-  override def requiredChildOrdering: Seq[Seq[SortOrder]] =
-    Seq(partitionSpec.map(SortOrder(_, Ascending)) ++ orderSpec)
-
-  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
-
-  /**
-   * Create a bound ordering object for a given frame type and offset. A bound ordering object is
-   * used to determine which input row lies within the frame boundaries of an output row.
-   *
-   * This method uses Code Generation. It can only be used on the executor side.
-   *
-   * @param frameType to evaluate. This can either be Row or Range based.
-   * @param offset with respect to the row.
-   * @return a bound ordering object.
-   */
-  private[this] def createBoundOrdering(frameType: FrameType, offset: Int): BoundOrdering = {
-    frameType match {
-      case RangeFrame =>
-        val (exprs, current, bound) = if (offset == 0) {
-          // Use the entire order expression when the offset is 0.
-          val exprs = orderSpec.map(_.child)
-          val buildProjection = () => newMutableProjection(exprs, child.output)
-          (orderSpec, buildProjection(), buildProjection())
-        } else if (orderSpec.size == 1) {
-          // Use only the first order expression when the offset is non-null.
-          val sortExpr = orderSpec.head
-          val expr = sortExpr.child
-          // Create the projection which returns the current 'value'.
-          val current = newMutableProjection(expr :: Nil, child.output)
-          // Flip the sign of the offset when processing the order is descending
-          val boundOffset = sortExpr.direction match {
-            case Descending => -offset
-            case Ascending => offset
-          }
-          // Create the projection which returns the current 'value' modified by adding the offset.
-          val boundExpr = Add(expr, Cast(Literal.create(boundOffset, IntegerType), expr.dataType))
-          val bound = newMutableProjection(boundExpr :: Nil, child.output)
-          (sortExpr :: Nil, current, bound)
-        } else {
-          sys.error("Non-Zero range offsets are not supported for windows " +
-            "with multiple order expressions.")
-        }
-        // Construct the ordering. This is used to compare the result of current value projection
-        // to the result of bound value projection. This is done manually because we want to use
-        // Code Generation (if it is enabled).
-        val sortExprs = exprs.zipWithIndex.map { case (e, i) =>
-          SortOrder(BoundReference(i, e.dataType, e.nullable), e.direction)
-        }
-        val ordering = newOrdering(sortExprs, Nil)
-        RangeBoundOrdering(ordering, current, bound)
-      case RowFrame => RowBoundOrdering(offset)
-    }
-  }
-
-  /**
-   * Collection containing an entry for each window frame to process. Each entry contains a frames'
-   * WindowExpressions and factory function for the WindowFrameFunction.
-   */
-  private[this] lazy val windowFrameExpressionFactoryPairs = {
-    type FrameKey = (String, FrameType, Option[Int], Option[Int])
-    type ExpressionBuffer = mutable.Buffer[Expression]
-    val framedFunctions = mutable.Map.empty[FrameKey, (ExpressionBuffer, ExpressionBuffer)]
-
-    // Add a function and its function to the map for a given frame.
-    def collect(tpe: String, fr: SpecifiedWindowFrame, e: Expression, fn: Expression): Unit = {
-      val key = (tpe, fr.frameType, FrameBoundary(fr.frameStart), FrameBoundary(fr.frameEnd))
-      val (es, fns) = framedFunctions.getOrElseUpdate(
-        key, (ArrayBuffer.empty[Expression], ArrayBuffer.empty[Expression]))
-      es.append(e)
-      fns.append(fn)
-    }
-
-    // Collect all valid window functions and group them by their frame.
-    windowExpression.foreach { x =>
-      x.foreach {
-        case e @ WindowExpression(function, spec) =>
-          val frame = spec.frameSpecification.asInstanceOf[SpecifiedWindowFrame]
-          function match {
-            case AggregateExpression(f, _, _, _) => collect("AGGREGATE", frame, e, f)
-            case f: AggregateWindowFunction => collect("AGGREGATE", frame, e, f)
-            case f: OffsetWindowFunction => collect("OFFSET", frame, e, f)
-            case f => sys.error(s"Unsupported window function: $f")
-          }
-        case _ =>
-      }
-    }
-
-    // Map the groups to a (unbound) expression and frame factory pair.
-    var numExpressions = 0
-    framedFunctions.toSeq.map {
-      case (key, (expressions, functionSeq)) =>
-        val ordinal = numExpressions
-        val functions = functionSeq.toArray
-
-        // Construct an aggregate processor if we need one.
-        def processor = AggregateProcessor(
-          functions,
-          ordinal,
-          child.output,
-          (expressions, schema) =>
-            newMutableProjection(expressions, schema, subexpressionEliminationEnabled))
-
-        // Create the factory
-        val factory = key match {
-          // Offset Frame
-          case ("OFFSET", RowFrame, Some(offset), Some(h)) if offset == h =>
-            target: MutableRow =>
-              new OffsetWindowFunctionFrame(
-                target,
-                ordinal,
-                functions,
-                child.output,
-                (expressions, schema) =>
-                  newMutableProjection(expressions, schema, subexpressionEliminationEnabled),
-                offset)
-
-          // Growing Frame.
-          case ("AGGREGATE", frameType, None, Some(high)) =>
-            target: MutableRow => {
-              new UnboundedPrecedingWindowFunctionFrame(
-                target,
-                processor,
-                createBoundOrdering(frameType, high))
-            }
-
-          // Shrinking Frame.
-          case ("AGGREGATE", frameType, Some(low), None) =>
-            target: MutableRow => {
-              new UnboundedFollowingWindowFunctionFrame(
-                target,
-                processor,
-                createBoundOrdering(frameType, low))
-            }
-
-          // Moving Frame.
-          case ("AGGREGATE", frameType, Some(low), Some(high)) =>
-            target: MutableRow => {
-              new SlidingWindowFunctionFrame(
-                target,
-                processor,
-                createBoundOrdering(frameType, low),
-                createBoundOrdering(frameType, high))
-            }
-
-          // Entire Partition Frame.
-          case ("AGGREGATE", frameType, None, None) =>
-            target: MutableRow => {
-              new UnboundedWindowFunctionFrame(target, processor)
-            }
-        }
-
-        // Keep track of the number of expressions. This is a side-effect in a map...
-        numExpressions += expressions.size
-
-        // Create the Frame Expression - Factory pair.
-        (expressions, factory)
-    }
-  }
-
-  /**
-   * Create the resulting projection.
-   *
-   * This method uses Code Generation. It can only be used on the executor side.
-   *
-   * @param expressions unbound ordered function expressions.
-   * @return the final resulting projection.
-   */
-  private[this] def createResultProjection(
-      expressions: Seq[Expression]): UnsafeProjection = {
-    val references = expressions.zipWithIndex.map{ case (e, i) =>
-      // Results of window expressions will be on the right side of child's output
-      BoundReference(child.output.size + i, e.dataType, e.nullable)
-    }
-    val unboundToRefMap = expressions.zip(references).toMap
-    val patchedWindowExpression = windowExpression.map(_.transform(unboundToRefMap))
-    UnsafeProjection.create(
-      child.output ++ patchedWindowExpression,
-      child.output)
-  }
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    // Unwrap the expressions and factories from the map.
-    val expressions = windowFrameExpressionFactoryPairs.flatMap(_._1)
-    val factories = windowFrameExpressionFactoryPairs.map(_._2).toArray
-
-    // Start processing.
-    child.execute().mapPartitions { stream =>
-      new Iterator[InternalRow] {
-
-        // Get all relevant projections.
-        val result = createResultProjection(expressions)
-        val grouping = UnsafeProjection.create(partitionSpec, child.output)
-
-        // Manage the stream and the grouping.
-        var nextRow: UnsafeRow = null
-        var nextGroup: UnsafeRow = null
-        var nextRowAvailable: Boolean = false
-        private[this] def fetchNextRow() {
-          nextRowAvailable = stream.hasNext
-          if (nextRowAvailable) {
-            nextRow = stream.next().asInstanceOf[UnsafeRow]
-            nextGroup = grouping(nextRow)
-          } else {
-            nextRow = null
-            nextGroup = null
-          }
-        }
-        fetchNextRow()
-
-        // Manage the current partition.
-        val rows = ArrayBuffer.empty[UnsafeRow]
-        val inputFields = child.output.length
-        var sorter: UnsafeExternalSorter = null
-        var rowBuffer: RowBuffer = null
-        val windowFunctionResult = new SpecificMutableRow(expressions.map(_.dataType))
-        val frames = factories.map(_(windowFunctionResult))
-        val numFrames = frames.length
-        private[this] def fetchNextPartition() {
-          // Collect all the rows in the current partition.
-          // Before we start to fetch new input rows, make a copy of nextGroup.
-          val currentGroup = nextGroup.copy()
-
-          // clear last partition
-          if (sorter != null) {
-            // the last sorter of this task will be cleaned up via task completion listener
-            sorter.cleanupResources()
-            sorter = null
-          } else {
-            rows.clear()
-          }
-
-          while (nextRowAvailable && nextGroup == currentGroup) {
-            if (sorter == null) {
-              rows += nextRow.copy()
-
-              if (rows.length >= 4096) {
-                // We will not sort the rows, so prefixComparator and recordComparator are null.
-                sorter = UnsafeExternalSorter.create(
-                  TaskContext.get().taskMemoryManager(),
-                  SparkEnv.get.blockManager,
-                  SparkEnv.get.serializerManager,
-                  TaskContext.get(),
-                  null,
-                  null,
-                  1024,
-                  SparkEnv.get.memoryManager.pageSizeBytes,
-                  false)
-                rows.foreach { r =>
-                  sorter.insertRecord(r.getBaseObject, r.getBaseOffset, r.getSizeInBytes, 0)
-                }
-                rows.clear()
-              }
-            } else {
-              sorter.insertRecord(nextRow.getBaseObject, nextRow.getBaseOffset,
-                nextRow.getSizeInBytes, 0)
-            }
-            fetchNextRow()
-          }
-          if (sorter != null) {
-            rowBuffer = new ExternalRowBuffer(sorter, inputFields)
-          } else {
-            rowBuffer = new ArrayRowBuffer(rows)
-          }
-
-          // Setup the frames.
-          var i = 0
-          while (i < numFrames) {
-            frames(i).prepare(rowBuffer.copy())
-            i += 1
-          }
-
-          // Setup iteration
-          rowIndex = 0
-          rowsSize = rowBuffer.size()
-        }
-
-        // Iteration
-        var rowIndex = 0
-        var rowsSize = 0L
-
-        override final def hasNext: Boolean = rowIndex < rowsSize || nextRowAvailable
-
-        val join = new JoinedRow
-        override final def next(): InternalRow = {
-          // Load the next partition if we need to.
-          if (rowIndex >= rowsSize && nextRowAvailable) {
-            fetchNextPartition()
-          }
-
-          if (rowIndex < rowsSize) {
-            // Get the results for the window frames.
-            var i = 0
-            val current = rowBuffer.next()
-            while (i < numFrames) {
-              frames(i).write(rowIndex, current)
-              i += 1
-            }
-
-            // 'Merge' the input row with the window function result
-            join(current, windowFunctionResult)
-            rowIndex += 1
-
-            // Return the projection.
-            result(join)
-          } else throw new NoSuchElementException
-        }
-      }
-    }
-  }
-}
-
-/**
- * Function for comparing boundary values.
- */
-private[execution] abstract class BoundOrdering {
-  def compare(inputRow: InternalRow, inputIndex: Int, outputRow: InternalRow, outputIndex: Int): Int
-}
-
-/**
- * Compare the input index to the bound of the output index.
- */
-private[execution] final case class RowBoundOrdering(offset: Int) extends BoundOrdering {
-  override def compare(
-      inputRow: InternalRow,
-      inputIndex: Int,
-      outputRow: InternalRow,
-      outputIndex: Int): Int =
-    inputIndex - (outputIndex + offset)
-}
-
-/**
- * Compare the value of the input index to the value bound of the output index.
- */
-private[execution] final case class RangeBoundOrdering(
-    ordering: Ordering[InternalRow],
-    current: Projection,
-    bound: Projection) extends BoundOrdering {
-  override def compare(
-      inputRow: InternalRow,
-      inputIndex: Int,
-      outputRow: InternalRow,
-      outputIndex: Int): Int =
-    ordering.compare(current(inputRow), bound(outputRow))
-}
-
-/**
- * The interface of row buffer for a partition
- */
-private[execution] abstract class RowBuffer {
-
-  /** Number of rows. */
-  def size(): Int
-
-  /** Return next row in the buffer, null if no more left. */
-  def next(): InternalRow
-
-  /** Skip the next `n` rows. */
-  def skip(n: Int): Unit
-
-  /** Return a new RowBuffer that has the same rows. */
-  def copy(): RowBuffer
-}
-
-/**
- * A row buffer based on ArrayBuffer (the number of rows is limited)
- */
-private[execution] class ArrayRowBuffer(buffer: ArrayBuffer[UnsafeRow]) extends RowBuffer {
-
-  private[this] var cursor: Int = -1
-
-  /** Number of rows. */
-  def size(): Int = buffer.length
-
-  /** Return next row in the buffer, null if no more left. */
-  def next(): InternalRow = {
-    cursor += 1
-    if (cursor < buffer.length) {
-      buffer(cursor)
-    } else {
-      null
-    }
-  }
-
-  /** Skip the next `n` rows. */
-  def skip(n: Int): Unit = {
-    cursor += n
-  }
-
-  /** Return a new RowBuffer that has the same rows. */
-  def copy(): RowBuffer = {
-    new ArrayRowBuffer(buffer)
-  }
-}
-
-/**
- * An external buffer of rows based on UnsafeExternalSorter
- */
-private[execution] class ExternalRowBuffer(sorter: UnsafeExternalSorter, numFields: Int)
-  extends RowBuffer {
-
-  private[this] val iter: UnsafeSorterIterator = sorter.getIterator
-
-  private[this] val currentRow = new UnsafeRow(numFields)
-
-  /** Number of rows. */
-  def size(): Int = iter.getNumRecords()
-
-  /** Return next row in the buffer, null if no more left. */
-  def next(): InternalRow = {
-    if (iter.hasNext) {
-      iter.loadNext()
-      currentRow.pointTo(iter.getBaseObject, iter.getBaseOffset, iter.getRecordLength)
-      currentRow
-    } else {
-      null
-    }
-  }
-
-  /** Skip the next `n` rows. */
-  def skip(n: Int): Unit = {
-    var i = 0
-    while (i < n && iter.hasNext) {
-      iter.loadNext()
-      i += 1
-    }
-  }
-
-  /** Return a new RowBuffer that has the same rows. */
-  def copy(): RowBuffer = {
-    new ExternalRowBuffer(sorter, numFields)
-  }
-}
-
-/**
- * A window function calculates the results of a number of window functions for a window frame.
- * Before use a frame must be prepared by passing it all the rows in the current partition. After
- * preparation the update method can be called to fill the output rows.
- */
-private[execution] abstract class WindowFunctionFrame {
-  /**
-   * Prepare the frame for calculating the results for a partition.
-   *
-   * @param rows to calculate the frame results for.
-   */
-  def prepare(rows: RowBuffer): Unit
-
-  /**
-   * Write the current results to the target row.
-   */
-  def write(index: Int, current: InternalRow): Unit
-}
-
-/**
- * The offset window frame calculates frames containing LEAD/LAG statements.
- *
- * @param target to write results to.
- * @param expressions to shift a number of rows.
- * @param inputSchema required for creating a projection.
- * @param newMutableProjection function used to create the projection.
- * @param offset by which rows get moved within a partition.
- */
-private[execution] final class OffsetWindowFunctionFrame(
-    target: MutableRow,
-    ordinal: Int,
-    expressions: Array[Expression],
-    inputSchema: Seq[Attribute],
-    newMutableProjection: (Seq[Expression], Seq[Attribute]) => MutableProjection,
-    offset: Int) extends WindowFunctionFrame {
-
-  /** Rows of the partition currently being processed. */
-  private[this] var input: RowBuffer = null
-
-  /** Index of the input row currently used for output. */
-  private[this] var inputIndex = 0
-
-  /** Row used when there is no valid input. */
-  private[this] val emptyRow = new GenericInternalRow(inputSchema.size)
-
-  /** Row used to combine the offset and the current row. */
-  private[this] val join = new JoinedRow
-
-  /** Create the projection. */
-  private[this] val projection = {
-    // Collect the expressions and bind them.
-    val inputAttrs = inputSchema.map(_.withNullability(true))
-    val numInputAttributes = inputAttrs.size
-    val boundExpressions = Seq.fill(ordinal)(NoOp) ++ expressions.toSeq.map {
-      case e: OffsetWindowFunction =>
-        val input = BindReferences.bindReference(e.input, inputAttrs)
-        if (e.default == null || e.default.foldable && e.default.eval() == null) {
-          // Without default value.
-          input
-        } else {
-          // With default value.
-          val default = BindReferences.bindReference(e.default, inputAttrs).transform {
-            // Shift the input reference to its default version.
-            case BoundReference(o, dataType, nullable) =>
-              BoundReference(o + numInputAttributes, dataType, nullable)
-          }
-          org.apache.spark.sql.catalyst.expressions.Coalesce(input :: default :: Nil)
-        }
-      case e =>
-        BindReferences.bindReference(e, inputAttrs)
-    }
-
-    // Create the projection.
-    newMutableProjection(boundExpressions, Nil).target(target)
-  }
-
-  override def prepare(rows: RowBuffer): Unit = {
-    input = rows
-    // drain the first few rows if offset is larger than zero
-    inputIndex = 0
-    while (inputIndex < offset) {
-      input.next()
-      inputIndex += 1
-    }
-    inputIndex = offset
-  }
-
-  override def write(index: Int, current: InternalRow): Unit = {
-    if (inputIndex >= 0 && inputIndex < input.size) {
-      val r = input.next()
-      join(r, current)
-    } else {
-      join(emptyRow, current)
-    }
-    projection(join)
-    inputIndex += 1
-  }
-}
-
-/**
- * The sliding window frame calculates frames with the following SQL form:
- * ... BETWEEN 1 PRECEDING AND 1 FOLLOWING
- *
- * @param target to write results to.
- * @param processor to calculate the row values with.
- * @param lbound comparator used to identify the lower bound of an output row.
- * @param ubound comparator used to identify the upper bound of an output row.
- */
-private[execution] final class SlidingWindowFunctionFrame(
-    target: MutableRow,
-    processor: AggregateProcessor,
-    lbound: BoundOrdering,
-    ubound: BoundOrdering) extends WindowFunctionFrame {
-
-  /** Rows of the partition currently being processed. */
-  private[this] var input: RowBuffer = null
-
-  /** The next row from `input`. */
-  private[this] var nextRow: InternalRow = null
-
-  /** The rows within current sliding window. */
-  private[this] val buffer = new util.ArrayDeque[InternalRow]()
-
-  /**
-   * Index of the first input row with a value greater than the upper bound of the current
-   * output row.
-   */
-  private[this] var inputHighIndex = 0
-
-  /**
-   * Index of the first input row with a value equal to or greater than the lower bound of the
-   * current output row.
-   */
-  private[this] var inputLowIndex = 0
-
-  /** Prepare the frame for calculating a new partition. Reset all variables. */
-  override def prepare(rows: RowBuffer): Unit = {
-    input = rows
-    nextRow = rows.next()
-    inputHighIndex = 0
-    inputLowIndex = 0
-    buffer.clear()
-  }
-
-  /** Write the frame columns for the current row to the given target row. */
-  override def write(index: Int, current: InternalRow): Unit = {
-    var bufferUpdated = index == 0
-
-    // Add all rows to the buffer for which the input row value is equal to or less than
-    // the output row upper bound.
-    while (nextRow != null && ubound.compare(nextRow, inputHighIndex, current, index) <= 0) {
-      buffer.add(nextRow.copy())
-      nextRow = input.next()
-      inputHighIndex += 1
-      bufferUpdated = true
-    }
-
-    // Drop all rows from the buffer for which the input row value is smaller than
-    // the output row lower bound.
-    while (!buffer.isEmpty && lbound.compare(buffer.peek(), inputLowIndex, current, index) < 0) {
-      buffer.remove()
-      inputLowIndex += 1
-      bufferUpdated = true
-    }
-
-    // Only recalculate and update when the buffer changes.
-    if (bufferUpdated) {
-      processor.initialize(input.size)
-      val iter = buffer.iterator()
-      while (iter.hasNext) {
-        processor.update(iter.next())
-      }
-      processor.evaluate(target)
-    }
-  }
-}
-
-/**
- * The unbounded window frame calculates frames with the following SQL forms:
- * ... (No Frame Definition)
- * ... BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
- *
- * Its results are  the same for each and every row in the partition. This class can be seen as a
- * special case of a sliding window, but is optimized for the unbound case.
- *
- * @param target to write results to.
- * @param processor to calculate the row values with.
- */
-private[execution] final class UnboundedWindowFunctionFrame(
-    target: MutableRow,
-    processor: AggregateProcessor) extends WindowFunctionFrame {
-
-  /** Prepare the frame for calculating a new partition. Process all rows eagerly. */
-  override def prepare(rows: RowBuffer): Unit = {
-    val size = rows.size()
-    processor.initialize(size)
-    var i = 0
-    while (i < size) {
-      processor.update(rows.next())
-      i += 1
-    }
-  }
-
-  /** Write the frame columns for the current row to the given target row. */
-  override def write(index: Int, current: InternalRow): Unit = {
-    // Unfortunately we cannot assume that evaluation is deterministic. So we need to re-evaluate
-    // for each row.
-    processor.evaluate(target)
-  }
-}
-
-/**
- * The UnboundPreceding window frame calculates frames with the following SQL form:
- * ... BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
- *
- * There is only an upper bound. Very common use cases are for instance running sums or counts
- * (row_number). Technically this is a special case of a sliding window. However a sliding window
- * has to maintain a buffer, and it must do a full evaluation everytime the buffer changes. This
- * is not the case when there is no lower bound, given the additive nature of most aggregates
- * streaming updates and partial evaluation suffice and no buffering is needed.
- *
- * @param target to write results to.
- * @param processor to calculate the row values with.
- * @param ubound comparator used to identify the upper bound of an output row.
- */
-private[execution] final class UnboundedPrecedingWindowFunctionFrame(
-    target: MutableRow,
-    processor: AggregateProcessor,
-    ubound: BoundOrdering) extends WindowFunctionFrame {
-
-  /** Rows of the partition currently being processed. */
-  private[this] var input: RowBuffer = null
-
-  /** The next row from `input`. */
-  private[this] var nextRow: InternalRow = null
-
-  /**
-   * Index of the first input row with a value greater than the upper bound of the current
-   * output row.
-   */
-  private[this] var inputIndex = 0
-
-  /** Prepare the frame for calculating a new partition. */
-  override def prepare(rows: RowBuffer): Unit = {
-    input = rows
-    nextRow = rows.next()
-    inputIndex = 0
-    processor.initialize(input.size)
-  }
-
-  /** Write the frame columns for the current row to the given target row. */
-  override def write(index: Int, current: InternalRow): Unit = {
-    var bufferUpdated = index == 0
-
-    // Add all rows to the aggregates for which the input row value is equal to or less than
-    // the output row upper bound.
-    while (nextRow != null && ubound.compare(nextRow, inputIndex, current, index) <= 0) {
-      processor.update(nextRow)
-      nextRow = input.next()
-      inputIndex += 1
-      bufferUpdated = true
-    }
-
-    // Only recalculate and update when the buffer changes.
-    if (bufferUpdated) {
-      processor.evaluate(target)
-    }
-  }
-}
-
-/**
- * The UnboundFollowing window frame calculates frames with the following SQL form:
- * ... BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING
- *
- * There is only an upper bound. This is a slightly modified version of the sliding window. The
- * sliding window operator has to check if both upper and the lower bound change when a new row
- * gets processed, where as the unbounded following only has to check the lower bound.
- *
- * This is a very expensive operator to use, O(n * (n - 1) /2), because we need to maintain a
- * buffer and must do full recalculation after each row. Reverse iteration would be possible, if
- * the commutativity of the used window functions can be guaranteed.
- *
- * @param target to write results to.
- * @param processor to calculate the row values with.
- * @param lbound comparator used to identify the lower bound of an output row.
- */
-private[execution] final class UnboundedFollowingWindowFunctionFrame(
-    target: MutableRow,
-    processor: AggregateProcessor,
-    lbound: BoundOrdering) extends WindowFunctionFrame {
-
-  /** Rows of the partition currently being processed. */
-  private[this] var input: RowBuffer = null
-
-  /**
-   * Index of the first input row with a value equal to or greater than the lower bound of the
-   * current output row.
-   */
-  private[this] var inputIndex = 0
-
-  /** Prepare the frame for calculating a new partition. */
-  override def prepare(rows: RowBuffer): Unit = {
-    input = rows
-    inputIndex = 0
-  }
-
-  /** Write the frame columns for the current row to the given target row. */
-  override def write(index: Int, current: InternalRow): Unit = {
-    var bufferUpdated = index == 0
-
-    // Duplicate the input to have a new iterator
-    val tmp = input.copy()
-
-    // Drop all rows from the buffer for which the input row value is smaller than
-    // the output row lower bound.
-    tmp.skip(inputIndex)
-    var nextRow = tmp.next()
-    while (nextRow != null && lbound.compare(nextRow, inputIndex, current, index) < 0) {
-      nextRow = tmp.next()
-      inputIndex += 1
-      bufferUpdated = true
-    }
-
-    // Only recalculate and update when the buffer changes.
-    if (bufferUpdated) {
-      processor.initialize(input.size)
-      while (nextRow != null) {
-        processor.update(nextRow)
-        nextRow = tmp.next()
-      }
-      processor.evaluate(target)
-    }
-  }
-}
-
-/**
- * This class prepares and manages the processing of a number of [[AggregateFunction]]s within a
- * single frame. The [[WindowFunctionFrame]] takes care of processing the frame in the correct way,
- * this reduces the processing of a [[AggregateWindowFunction]] to processing the underlying
- * [[AggregateFunction]]. All [[AggregateFunction]]s are processed in [[Complete]] mode.
- *
- * [[SizeBasedWindowFunction]]s are initialized in a slightly different way. These functions
- * require the size of the partition processed, this value is exposed to them when the processor is
- * constructed.
- *
- * Processing of distinct aggregates is currently not supported.
- *
- * The implementation is split into an object which takes care of construction, and a the actual
- * processor class.
- */
-private[execution] object AggregateProcessor {
-  def apply(
-      functions: Array[Expression],
-      ordinal: Int,
-      inputAttributes: Seq[Attribute],
-      newMutableProjection: (Seq[Expression], Seq[Attribute]) => MutableProjection):
-      AggregateProcessor = {
-    val aggBufferAttributes = mutable.Buffer.empty[AttributeReference]
-    val initialValues = mutable.Buffer.empty[Expression]
-    val updateExpressions = mutable.Buffer.empty[Expression]
-    val evaluateExpressions = mutable.Buffer.fill[Expression](ordinal)(NoOp)
-    val imperatives = mutable.Buffer.empty[ImperativeAggregate]
-
-    // SPARK-14244: `SizeBasedWindowFunction`s are firstly created on driver side and then
-    // serialized to executor side. These functions all reference a global singleton window
-    // partition size attribute reference, i.e., `SizeBasedWindowFunction.n`. Here we must collect
-    // the singleton instance created on driver side instead of using executor side
-    // `SizeBasedWindowFunction.n` to avoid binding failure caused by mismatching expression ID.
-    val partitionSize: Option[AttributeReference] = {
-      val aggs = functions.flatMap(_.collectFirst { case f: SizeBasedWindowFunction => f })
-      aggs.headOption.map(_.n)
-    }
-
-    // Check if there are any SizeBasedWindowFunctions. If there are, we add the partition size to
-    // the aggregation buffer. Note that the ordinal of the partition size value will always be 0.
-    partitionSize.foreach { n =>
-      aggBufferAttributes += n
-      initialValues += NoOp
-      updateExpressions += NoOp
-    }
-
-    // Add an AggregateFunction to the AggregateProcessor.
-    functions.foreach {
-      case agg: DeclarativeAggregate =>
-        aggBufferAttributes ++= agg.aggBufferAttributes
-        initialValues ++= agg.initialValues
-        updateExpressions ++= agg.updateExpressions
-        evaluateExpressions += agg.evaluateExpression
-      case agg: ImperativeAggregate =>
-        val offset = aggBufferAttributes.size
-        val imperative = BindReferences.bindReference(agg
-          .withNewInputAggBufferOffset(offset)
-          .withNewMutableAggBufferOffset(offset),
-          inputAttributes)
-        imperatives += imperative
-        aggBufferAttributes ++= imperative.aggBufferAttributes
-        val noOps = Seq.fill(imperative.aggBufferAttributes.size)(NoOp)
-        initialValues ++= noOps
-        updateExpressions ++= noOps
-        evaluateExpressions += imperative
-      case other =>
-        sys.error(s"Unsupported Aggregate Function: $other")
-    }
-
-    // Create the projections.
-    val initialProjection = newMutableProjection(
-      initialValues,
-      partitionSize.toSeq)
-    val updateProjection = newMutableProjection(
-      updateExpressions,
-      aggBufferAttributes ++ inputAttributes)
-    val evaluateProjection = newMutableProjection(
-      evaluateExpressions,
-      aggBufferAttributes)
-
-    // Create the processor
-    new AggregateProcessor(
-      aggBufferAttributes.toArray,
-      initialProjection,
-      updateProjection,
-      evaluateProjection,
-      imperatives.toArray,
-      partitionSize.isDefined)
-  }
-}
-
-/**
- * This class manages the processing of a number of aggregate functions. See the documentation of
- * the object for more information.
- */
-private[execution] final class AggregateProcessor(
-    private[this] val bufferSchema: Array[AttributeReference],
-    private[this] val initialProjection: MutableProjection,
-    private[this] val updateProjection: MutableProjection,
-    private[this] val evaluateProjection: MutableProjection,
-    private[this] val imperatives: Array[ImperativeAggregate],
-    private[this] val trackPartitionSize: Boolean) {
-
-  private[this] val join = new JoinedRow
-  private[this] val numImperatives = imperatives.length
-  private[this] val buffer = new SpecificMutableRow(bufferSchema.toSeq.map(_.dataType))
-  initialProjection.target(buffer)
-  updateProjection.target(buffer)
-
-  /** Create the initial state. */
-  def initialize(size: Int): Unit = {
-    // Some initialization expressions are dependent on the partition size so we have to
-    // initialize the size before initializing all other fields, and we have to pass the buffer to
-    // the initialization projection.
-    if (trackPartitionSize) {
-      buffer.setInt(0, size)
-    }
-    initialProjection(buffer)
-    var i = 0
-    while (i < numImperatives) {
-      imperatives(i).initialize(buffer)
-      i += 1
-    }
-  }
-
-  /** Update the buffer. */
-  def update(input: InternalRow): Unit = {
-    updateProjection(join(buffer, input))
-    var i = 0
-    while (i < numImperatives) {
-      imperatives(i).update(buffer, input)
-      i += 1
-    }
-  }
-
-  /** Evaluate buffer. */
-  def evaluate(target: MutableRow): Unit =
-    evaluateProjection.target(target)(buffer)
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[7/7] spark git commit: [SPARK-14855][SQL] Add "Exec" suffix to physical operators

Posted by rx...@apache.org.
[SPARK-14855][SQL] Add "Exec" suffix to physical operators

## What changes were proposed in this pull request?
This patch adds "Exec" suffix to all physical operators. Before this patch, Spark's physical operators and logical operators are named the same (e.g. Project could be logical.Project or execution.Project), which caused small issues in code review and bigger issues in code refactoring.

## How was this patch tested?
N/A

Author: Reynold Xin <rx...@databricks.com>

Closes #12617 from rxin/exec-node.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d7d0cad0
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d7d0cad0
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d7d0cad0

Branch: refs/heads/master
Commit: d7d0cad0ad7667c0e09ae01601ee0e4d0b09963c
Parents: c431a76
Author: Reynold Xin <rx...@databricks.com>
Authored: Fri Apr 22 17:43:56 2016 -0700
Committer: Reynold Xin <rx...@databricks.com>
Committed: Fri Apr 22 17:43:56 2016 -0700

----------------------------------------------------------------------
 .../spark/sql/catalyst/trees/TreeNode.scala     |    9 +-
 .../spark/sql/execution/ExistingRDD.scala       |   24 +-
 .../org/apache/spark/sql/execution/Expand.scala |  202 ----
 .../apache/spark/sql/execution/ExpandExec.scala |  202 ++++
 .../apache/spark/sql/execution/Generate.scala   |  103 --
 .../spark/sql/execution/GenerateExec.scala      |  103 ++
 .../spark/sql/execution/LocalTableScan.scala    |   58 -
 .../sql/execution/LocalTableScanExec.scala      |   58 +
 .../spark/sql/execution/QueryExecution.scala    |    6 +-
 .../org/apache/spark/sql/execution/Sort.scala   |  183 ----
 .../apache/spark/sql/execution/SortExec.scala   |  183 ++++
 .../apache/spark/sql/execution/SparkPlan.scala  |   10 +-
 .../spark/sql/execution/SparkPlanInfo.scala     |    4 +-
 .../spark/sql/execution/SparkPlanner.scala      |    4 +-
 .../spark/sql/execution/SparkStrategies.scala   |  110 +-
 .../spark/sql/execution/WholeStageCodegen.scala |  492 ---------
 .../sql/execution/WholeStageCodegenExec.scala   |  492 +++++++++
 .../org/apache/spark/sql/execution/Window.scala | 1008 ------------------
 .../apache/spark/sql/execution/WindowExec.scala | 1008 ++++++++++++++++++
 .../aggregate/SortBasedAggregate.scala          |  111 --
 .../aggregate/SortBasedAggregateExec.scala      |  111 ++
 .../execution/aggregate/TungstenAggregate.scala |    2 +-
 .../spark/sql/execution/aggregate/utils.scala   |   10 +-
 .../spark/sql/execution/basicOperators.scala    |   50 +-
 .../columnar/InMemoryColumnarTableScan.scala    |  358 -------
 .../columnar/InMemoryTableScanExec.scala        |  358 +++++++
 .../spark/sql/execution/command/commands.scala  |    2 +-
 .../datasources/DataSourceStrategy.scala        |   18 +-
 .../datasources/FileSourceStrategy.scala        |    8 +-
 .../spark/sql/execution/debug/package.scala     |   10 +-
 .../execution/exchange/BroadcastExchange.scala  |  109 --
 .../exchange/BroadcastExchangeExec.scala        |  109 ++
 .../execution/exchange/EnsureRequirements.scala |    4 +-
 .../spark/sql/execution/exchange/Exchange.scala |    9 +-
 .../sql/execution/joins/BroadcastHashJoin.scala |  401 -------
 .../execution/joins/BroadcastHashJoinExec.scala |  401 +++++++
 .../joins/BroadcastNestedLoopJoin.scala         |  331 ------
 .../joins/BroadcastNestedLoopJoinExec.scala     |  331 ++++++
 .../sql/execution/joins/CartesianProduct.scala  |  103 --
 .../execution/joins/CartesianProductExec.scala  |  103 ++
 .../sql/execution/joins/ShuffledHashJoin.scala  |   81 --
 .../execution/joins/ShuffledHashJoinExec.scala  |   81 ++
 .../sql/execution/joins/SortMergeJoin.scala     |  964 -----------------
 .../sql/execution/joins/SortMergeJoinExec.scala |  964 +++++++++++++++++
 .../org/apache/spark/sql/execution/limit.scala  |   19 +-
 .../apache/spark/sql/execution/objects.scala    |   39 +-
 .../execution/python/BatchEvalPythonExec.scala  |  149 +++
 .../python/BatchPythonEvaluation.scala          |  149 ---
 .../execution/python/ExtractPythonUDFs.scala    |    4 +-
 .../streaming/IncrementalExecution.scala        |   15 +-
 .../execution/streaming/StatefulAggregate.scala |   10 +-
 .../apache/spark/sql/execution/subquery.scala   |    2 +-
 .../spark/sql/execution/ui/SparkPlanGraph.scala |    4 +-
 .../org/apache/spark/sql/CachedTableSuite.scala |   10 +-
 .../spark/sql/ColumnExpressionSuite.scala       |    4 +-
 .../apache/spark/sql/DataFrameJoinSuite.scala   |    6 +-
 .../org/apache/spark/sql/DataFrameSuite.scala   |   10 +-
 .../scala/org/apache/spark/sql/JoinSuite.scala  |  115 +-
 .../org/apache/spark/sql/SQLQuerySuite.scala    |    8 +-
 .../spark/sql/execution/ExchangeSuite.scala     |   12 +-
 .../spark/sql/execution/PlannerSuite.scala      |   36 +-
 .../spark/sql/execution/ReferenceSort.scala     |    2 +-
 .../apache/spark/sql/execution/SortSuite.scala  |   16 +-
 .../execution/TakeOrderedAndProjectSuite.scala  |   21 +-
 .../sql/execution/WholeStageCodegenSuite.scala  |   36 +-
 .../columnar/PartitionBatchPruningSuite.scala   |    2 +-
 .../datasources/FileSourceStrategySuite.scala   |    6 +-
 .../execution/joins/BroadcastJoinSuite.scala    |    6 +-
 .../execution/joins/ExistenceJoinSuite.scala    |    8 +-
 .../sql/execution/joins/InnerJoinSuite.scala    |   14 +-
 .../sql/execution/joins/OuterJoinSuite.scala    |   10 +-
 .../org/apache/spark/sql/jdbc/JDBCSuite.scala   |   16 +-
 .../spark/sql/sources/FilteredScanSuite.scala   |    2 +-
 .../spark/sql/sources/PrunedScanSuite.scala     |    2 +-
 .../apache/spark/sql/test/SQLTestUtils.scala    |    4 +-
 .../spark/sql/util/DataFrameCallbackSuite.scala |    4 +-
 .../thriftserver/HiveThriftServer2Suites.scala  |    4 +-
 .../apache/spark/sql/hive/HiveStrategies.scala  |    8 +-
 .../sql/hive/execution/HiveTableScan.scala      |  166 ---
 .../sql/hive/execution/HiveTableScanExec.scala  |  166 +++
 .../hive/execution/InsertIntoHiveTable.scala    |    4 +-
 .../hive/execution/ScriptTransformation.scala   |    2 +-
 .../spark/sql/hive/CachedTableSuite.scala       |    6 +-
 .../apache/spark/sql/hive/StatisticsSuite.scala |   12 +-
 .../sql/hive/execution/HiveComparisonTest.scala |    2 +-
 .../sql/hive/execution/HiveQuerySuite.scala     |    4 +-
 .../hive/execution/HiveTypeCoercionSuite.scala  |    4 +-
 .../spark/sql/hive/execution/PruningSuite.scala |    2 +-
 .../execution/ScriptTransformationSuite.scala   |    4 +-
 .../apache/spark/sql/hive/parquetSuites.scala   |   14 +-
 .../spark/sql/sources/BucketedReadSuite.scala   |   10 +-
 .../sources/ParquetHadoopFsRelationSuite.scala  |    4 +-
 .../sql/sources/hadoopFsRelationSuites.scala    |    4 +-
 93 files changed, 5241 insertions(+), 5204 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
----------------------------------------------------------------------
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 232ca43..3d0e016 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

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
----------------------------------------------------------------------
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 12d03a7..b3a197c 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)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala
----------------------------------------------------------------------
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/Expand.scala
deleted file mode 100644
index 3966af5..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala
+++ /dev/null
@@ -1,202 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.errors._
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
-import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning}
-import org.apache.spark.sql.execution.metric.SQLMetrics
-
-/**
- * Apply the all of the GroupExpressions to every input row, hence we will get
- * multiple output rows for a input row.
- * @param projections The group of expressions, all of the group expressions should
- *                    output the same schema specified bye the parameter `output`
- * @param output      The output Schema
- * @param child       Child operator
- */
-case class Expand(
-    projections: Seq[Seq[Expression]],
-    output: Seq[Attribute],
-    child: SparkPlan)
-  extends UnaryNode with CodegenSupport {
-
-  private[sql] override lazy val metrics = Map(
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
-
-  // The GroupExpressions can output data with arbitrary partitioning, so set it
-  // as UNKNOWN partitioning
-  override def outputPartitioning: Partitioning = UnknownPartitioning(0)
-
-  override def references: AttributeSet =
-    AttributeSet(projections.flatten.flatMap(_.references))
-
-  private[this] val projection =
-    (exprs: Seq[Expression]) => UnsafeProjection.create(exprs, child.output)
-
-  protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") {
-    val numOutputRows = longMetric("numOutputRows")
-
-    child.execute().mapPartitions { iter =>
-      val groups = projections.map(projection).toArray
-      new Iterator[InternalRow] {
-        private[this] var result: InternalRow = _
-        private[this] var idx = -1  // -1 means the initial state
-        private[this] var input: InternalRow = _
-
-        override final def hasNext: Boolean = (-1 < idx && idx < groups.length) || iter.hasNext
-
-        override final def next(): InternalRow = {
-          if (idx <= 0) {
-            // in the initial (-1) or beginning(0) of a new input row, fetch the next input tuple
-            input = iter.next()
-            idx = 0
-          }
-
-          result = groups(idx)(input)
-          idx += 1
-
-          if (idx == groups.length && iter.hasNext) {
-            idx = 0
-          }
-
-          numOutputRows += 1
-          result
-        }
-      }
-    }
-  }
-
-  override def inputRDDs(): Seq[RDD[InternalRow]] = {
-    child.asInstanceOf[CodegenSupport].inputRDDs()
-  }
-
-  protected override def doProduce(ctx: CodegenContext): String = {
-    child.asInstanceOf[CodegenSupport].produce(ctx, this)
-  }
-
-  override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
-    /*
-     * When the projections list looks like:
-     *   expr1A, exprB, expr1C
-     *   expr2A, exprB, expr2C
-     *   ...
-     *   expr(N-1)A, exprB, expr(N-1)C
-     *
-     * i.e. column A and C have different values for each output row, but column B stays constant.
-     *
-     * The generated code looks something like (note that B is only computed once in declaration):
-     *
-     * // part 1: declare all the columns
-     * colA = ...
-     * colB = ...
-     * colC = ...
-     *
-     * // part 2: code that computes the columns
-     * for (row = 0; row < N; row++) {
-     *   switch (row) {
-     *     case 0:
-     *       colA = ...
-     *       colC = ...
-     *     case 1:
-     *       colA = ...
-     *       colC = ...
-     *     ...
-     *     case N - 1:
-     *       colA = ...
-     *       colC = ...
-     *   }
-     *   // increment metrics and consume output values
-     * }
-     *
-     * We use a for loop here so we only includes one copy of the consume code and avoid code
-     * size explosion.
-     */
-
-    // Set input variables
-    ctx.currentVars = input
-
-    // Tracks whether a column has the same output for all rows.
-    // Size of sameOutput array should equal N.
-    // If sameOutput(i) is true, then the i-th column has the same value for all output rows given
-    // an input row.
-    val sameOutput: Array[Boolean] = output.indices.map { colIndex =>
-      projections.map(p => p(colIndex)).toSet.size == 1
-    }.toArray
-
-    // Part 1: declare variables for each column
-    // If a column has the same value for all output rows, then we also generate its computation
-    // right after declaration. Otherwise its value is computed in the part 2.
-    val outputColumns = output.indices.map { col =>
-      val firstExpr = projections.head(col)
-      if (sameOutput(col)) {
-        // This column is the same across all output rows. Just generate code for it here.
-        BindReferences.bindReference(firstExpr, child.output).genCode(ctx)
-      } else {
-        val isNull = ctx.freshName("isNull")
-        val value = ctx.freshName("value")
-        val code = s"""
-          |boolean $isNull = true;
-          |${ctx.javaType(firstExpr.dataType)} $value = ${ctx.defaultValue(firstExpr.dataType)};
-         """.stripMargin
-        ExprCode(code, isNull, value)
-      }
-    }
-
-    // Part 2: switch/case statements
-    val cases = projections.zipWithIndex.map { case (exprs, row) =>
-      var updateCode = ""
-      for (col <- exprs.indices) {
-        if (!sameOutput(col)) {
-          val ev = BindReferences.bindReference(exprs(col), child.output).genCode(ctx)
-          updateCode +=
-            s"""
-               |${ev.code}
-               |${outputColumns(col).isNull} = ${ev.isNull};
-               |${outputColumns(col).value} = ${ev.value};
-            """.stripMargin
-        }
-      }
-
-      s"""
-         |case $row:
-         |  ${updateCode.trim}
-         |  break;
-       """.stripMargin
-    }
-
-    val numOutput = metricTerm(ctx, "numOutputRows")
-    val i = ctx.freshName("i")
-    // these column have to declared before the loop.
-    val evaluate = evaluateVariables(outputColumns)
-    ctx.copyResult = true
-    s"""
-       |$evaluate
-       |for (int $i = 0; $i < ${projections.length}; $i ++) {
-       |  switch ($i) {
-       |    ${cases.mkString("\n").trim}
-       |  }
-       |  $numOutput.add(1);
-       |  ${consume(ctx, outputColumns)}
-       |}
-     """.stripMargin
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala
new file mode 100644
index 0000000..7c47566
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.errors._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning}
+import org.apache.spark.sql.execution.metric.SQLMetrics
+
+/**
+ * Apply the all of the GroupExpressions to every input row, hence we will get
+ * multiple output rows for a input row.
+ * @param projections The group of expressions, all of the group expressions should
+ *                    output the same schema specified bye the parameter `output`
+ * @param output      The output Schema
+ * @param child       Child operator
+ */
+case class ExpandExec(
+    projections: Seq[Seq[Expression]],
+    output: Seq[Attribute],
+    child: SparkPlan)
+  extends UnaryExecNode with CodegenSupport {
+
+  private[sql] override lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
+
+  // The GroupExpressions can output data with arbitrary partitioning, so set it
+  // as UNKNOWN partitioning
+  override def outputPartitioning: Partitioning = UnknownPartitioning(0)
+
+  override def references: AttributeSet =
+    AttributeSet(projections.flatten.flatMap(_.references))
+
+  private[this] val projection =
+    (exprs: Seq[Expression]) => UnsafeProjection.create(exprs, child.output)
+
+  protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") {
+    val numOutputRows = longMetric("numOutputRows")
+
+    child.execute().mapPartitions { iter =>
+      val groups = projections.map(projection).toArray
+      new Iterator[InternalRow] {
+        private[this] var result: InternalRow = _
+        private[this] var idx = -1  // -1 means the initial state
+        private[this] var input: InternalRow = _
+
+        override final def hasNext: Boolean = (-1 < idx && idx < groups.length) || iter.hasNext
+
+        override final def next(): InternalRow = {
+          if (idx <= 0) {
+            // in the initial (-1) or beginning(0) of a new input row, fetch the next input tuple
+            input = iter.next()
+            idx = 0
+          }
+
+          result = groups(idx)(input)
+          idx += 1
+
+          if (idx == groups.length && iter.hasNext) {
+            idx = 0
+          }
+
+          numOutputRows += 1
+          result
+        }
+      }
+    }
+  }
+
+  override def inputRDDs(): Seq[RDD[InternalRow]] = {
+    child.asInstanceOf[CodegenSupport].inputRDDs()
+  }
+
+  protected override def doProduce(ctx: CodegenContext): String = {
+    child.asInstanceOf[CodegenSupport].produce(ctx, this)
+  }
+
+  override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
+    /*
+     * When the projections list looks like:
+     *   expr1A, exprB, expr1C
+     *   expr2A, exprB, expr2C
+     *   ...
+     *   expr(N-1)A, exprB, expr(N-1)C
+     *
+     * i.e. column A and C have different values for each output row, but column B stays constant.
+     *
+     * The generated code looks something like (note that B is only computed once in declaration):
+     *
+     * // part 1: declare all the columns
+     * colA = ...
+     * colB = ...
+     * colC = ...
+     *
+     * // part 2: code that computes the columns
+     * for (row = 0; row < N; row++) {
+     *   switch (row) {
+     *     case 0:
+     *       colA = ...
+     *       colC = ...
+     *     case 1:
+     *       colA = ...
+     *       colC = ...
+     *     ...
+     *     case N - 1:
+     *       colA = ...
+     *       colC = ...
+     *   }
+     *   // increment metrics and consume output values
+     * }
+     *
+     * We use a for loop here so we only includes one copy of the consume code and avoid code
+     * size explosion.
+     */
+
+    // Set input variables
+    ctx.currentVars = input
+
+    // Tracks whether a column has the same output for all rows.
+    // Size of sameOutput array should equal N.
+    // If sameOutput(i) is true, then the i-th column has the same value for all output rows given
+    // an input row.
+    val sameOutput: Array[Boolean] = output.indices.map { colIndex =>
+      projections.map(p => p(colIndex)).toSet.size == 1
+    }.toArray
+
+    // Part 1: declare variables for each column
+    // If a column has the same value for all output rows, then we also generate its computation
+    // right after declaration. Otherwise its value is computed in the part 2.
+    val outputColumns = output.indices.map { col =>
+      val firstExpr = projections.head(col)
+      if (sameOutput(col)) {
+        // This column is the same across all output rows. Just generate code for it here.
+        BindReferences.bindReference(firstExpr, child.output).genCode(ctx)
+      } else {
+        val isNull = ctx.freshName("isNull")
+        val value = ctx.freshName("value")
+        val code = s"""
+          |boolean $isNull = true;
+          |${ctx.javaType(firstExpr.dataType)} $value = ${ctx.defaultValue(firstExpr.dataType)};
+         """.stripMargin
+        ExprCode(code, isNull, value)
+      }
+    }
+
+    // Part 2: switch/case statements
+    val cases = projections.zipWithIndex.map { case (exprs, row) =>
+      var updateCode = ""
+      for (col <- exprs.indices) {
+        if (!sameOutput(col)) {
+          val ev = BindReferences.bindReference(exprs(col), child.output).genCode(ctx)
+          updateCode +=
+            s"""
+               |${ev.code}
+               |${outputColumns(col).isNull} = ${ev.isNull};
+               |${outputColumns(col).value} = ${ev.value};
+            """.stripMargin
+        }
+      }
+
+      s"""
+         |case $row:
+         |  ${updateCode.trim}
+         |  break;
+       """.stripMargin
+    }
+
+    val numOutput = metricTerm(ctx, "numOutputRows")
+    val i = ctx.freshName("i")
+    // these column have to declared before the loop.
+    val evaluate = evaluateVariables(outputColumns)
+    ctx.copyResult = true
+    s"""
+       |$evaluate
+       |for (int $i = 0; $i < ${projections.length}; $i ++) {
+       |  switch ($i) {
+       |    ${cases.mkString("\n").trim}
+       |  }
+       |  $numOutput.add(1);
+       |  ${consume(ctx, outputColumns)}
+       |}
+     """.stripMargin
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
----------------------------------------------------------------------
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/Generate.scala
deleted file mode 100644
index 9938d21..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.execution.metric.SQLMetrics
-
-/**
- * For lazy computing, be sure the generator.terminate() called in the very last
- * TODO reusing the CompletionIterator?
- */
-private[execution] sealed case class LazyIterator(func: () => TraversableOnce[InternalRow])
-  extends Iterator[InternalRow] {
-
-  lazy val results = func().toIterator
-  override def hasNext: Boolean = results.hasNext
-  override def next(): InternalRow = results.next()
-}
-
-/**
- * Applies a [[Generator]] to a stream of input rows, combining the
- * output of each into a new stream of rows.  This operation is similar to a `flatMap` in functional
- * programming with one important additional feature, which allows the input rows to be joined with
- * their output.
- * @param generator the generator expression
- * @param join  when true, each output row is implicitly joined with the input tuple that produced
- *              it.
- * @param outer when true, each input row will be output at least once, even if the output of the
- *              given `generator` is empty. `outer` has no effect when `join` is false.
- * @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(
-    generator: Generator,
-    join: Boolean,
-    outer: Boolean,
-    output: Seq[Attribute],
-    child: SparkPlan)
-  extends UnaryNode {
-
-  private[sql] override lazy val metrics = Map(
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
-
-  override def producedAttributes: AttributeSet = AttributeSet(output)
-
-  val boundGenerator = BindReferences.bindReference(generator, child.output)
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    // boundGenerator.terminate() should be triggered after all of the rows in the partition
-    val rows = if (join) {
-      child.execute().mapPartitionsInternal { iter =>
-        val generatorNullRow = new GenericInternalRow(generator.elementTypes.size)
-        val joinedRow = new JoinedRow
-
-        iter.flatMap { row =>
-          // we should always set the left (child output)
-          joinedRow.withLeft(row)
-          val outputRows = boundGenerator.eval(row)
-          if (outer && outputRows.isEmpty) {
-            joinedRow.withRight(generatorNullRow) :: Nil
-          } else {
-            outputRows.map(joinedRow.withRight)
-          }
-        } ++ LazyIterator(boundGenerator.terminate).map { row =>
-          // we leave the left side as the last element of its child output
-          // keep it the same as Hive does
-          joinedRow.withRight(row)
-        }
-      }
-    } else {
-      child.execute().mapPartitionsInternal { iter =>
-        iter.flatMap(boundGenerator.eval) ++ LazyIterator(boundGenerator.terminate)
-      }
-    }
-
-    val numOutputRows = longMetric("numOutputRows")
-    rows.mapPartitionsInternal { iter =>
-      val proj = UnsafeProjection.create(output, output)
-      iter.map { r =>
-        numOutputRows += 1
-        proj(r)
-      }
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala
new file mode 100644
index 0000000..10cfec3
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.metric.SQLMetrics
+
+/**
+ * For lazy computing, be sure the generator.terminate() called in the very last
+ * TODO reusing the CompletionIterator?
+ */
+private[execution] sealed case class LazyIterator(func: () => TraversableOnce[InternalRow])
+  extends Iterator[InternalRow] {
+
+  lazy val results = func().toIterator
+  override def hasNext: Boolean = results.hasNext
+  override def next(): InternalRow = results.next()
+}
+
+/**
+ * Applies a [[Generator]] to a stream of input rows, combining the
+ * output of each into a new stream of rows.  This operation is similar to a `flatMap` in functional
+ * programming with one important additional feature, which allows the input rows to be joined with
+ * their output.
+ * @param generator the generator expression
+ * @param join  when true, each output row is implicitly joined with the input tuple that produced
+ *              it.
+ * @param outer when true, each input row will be output at least once, even if the output of the
+ *              given `generator` is empty. `outer` has no effect when `join` is false.
+ * @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 GenerateExec(
+    generator: Generator,
+    join: Boolean,
+    outer: Boolean,
+    output: Seq[Attribute],
+    child: SparkPlan)
+  extends UnaryExecNode {
+
+  private[sql] override lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
+
+  override def producedAttributes: AttributeSet = AttributeSet(output)
+
+  val boundGenerator = BindReferences.bindReference(generator, child.output)
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    // boundGenerator.terminate() should be triggered after all of the rows in the partition
+    val rows = if (join) {
+      child.execute().mapPartitionsInternal { iter =>
+        val generatorNullRow = new GenericInternalRow(generator.elementTypes.size)
+        val joinedRow = new JoinedRow
+
+        iter.flatMap { row =>
+          // we should always set the left (child output)
+          joinedRow.withLeft(row)
+          val outputRows = boundGenerator.eval(row)
+          if (outer && outputRows.isEmpty) {
+            joinedRow.withRight(generatorNullRow) :: Nil
+          } else {
+            outputRows.map(joinedRow.withRight)
+          }
+        } ++ LazyIterator(boundGenerator.terminate).map { row =>
+          // we leave the left side as the last element of its child output
+          // keep it the same as Hive does
+          joinedRow.withRight(row)
+        }
+      }
+    } else {
+      child.execute().mapPartitionsInternal { iter =>
+        iter.flatMap(boundGenerator.eval) ++ LazyIterator(boundGenerator.terminate)
+      }
+    }
+
+    val numOutputRows = longMetric("numOutputRows")
+    rows.mapPartitionsInternal { iter =>
+      val proj = UnsafeProjection.create(output, output)
+      iter.map { r =>
+        numOutputRows += 1
+        proj(r)
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala
----------------------------------------------------------------------
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/LocalTableScan.scala
deleted file mode 100644
index f8aec9e..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection}
-import org.apache.spark.sql.execution.metric.SQLMetrics
-
-
-/**
- * Physical plan node for scanning data from a local collection.
- */
-private[sql] case class LocalTableScan(
-    output: Seq[Attribute],
-    rows: Seq[InternalRow]) extends LeafNode {
-
-  private[sql] override lazy val metrics = Map(
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
-
-  private val unsafeRows: Array[InternalRow] = {
-    val proj = UnsafeProjection.create(output, output)
-    rows.map(r => proj(r).copy()).toArray
-  }
-
-  private lazy val rdd = sqlContext.sparkContext.parallelize(unsafeRows)
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    val numOutputRows = longMetric("numOutputRows")
-    rdd.map { r =>
-      numOutputRows += 1
-      r
-    }
-  }
-
-  override def executeCollect(): Array[InternalRow] = {
-    unsafeRows
-  }
-
-  override def executeTake(limit: Int): Array[InternalRow] = {
-    unsafeRows.take(limit)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
new file mode 100644
index 0000000..4ab447a
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection}
+import org.apache.spark.sql.execution.metric.SQLMetrics
+
+
+/**
+ * Physical plan node for scanning data from a local collection.
+ */
+private[sql] case class LocalTableScanExec(
+    output: Seq[Attribute],
+    rows: Seq[InternalRow]) extends LeafExecNode {
+
+  private[sql] override lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
+
+  private val unsafeRows: Array[InternalRow] = {
+    val proj = UnsafeProjection.create(output, output)
+    rows.map(r => proj(r).copy()).toArray
+  }
+
+  private lazy val rdd = sqlContext.sparkContext.parallelize(unsafeRows)
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    val numOutputRows = longMetric("numOutputRows")
+    rdd.map { r =>
+      numOutputRows += 1
+      r
+    }
+  }
+
+  override def executeCollect(): Array[InternalRow] = {
+    unsafeRows
+  }
+
+  override def executeTake(limit: Int): Array[InternalRow] = {
+    unsafeRows.take(limit)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
----------------------------------------------------------------------
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 a444a70..bb83676 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 =>

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/Sort.scala
----------------------------------------------------------------------
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/Sort.scala
deleted file mode 100644
index 04a39a1..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Sort.scala
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution
-
-import org.apache.spark.{SparkEnv, TaskContext}
-import org.apache.spark.executor.TaskMetrics
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, GenerateUnsafeProjection}
-import org.apache.spark.sql.catalyst.plans.physical.{Distribution, OrderedDistribution, UnspecifiedDistribution}
-import org.apache.spark.sql.execution.metric.SQLMetrics
-import org.apache.spark.sql.types._
-import org.apache.spark.util.collection.unsafe.sort.RadixSort;
-
-/**
- * Performs (external) sorting.
- *
- * @param global when true performs a global sort of all partitions by shuffling the data first
- *               if necessary.
- * @param testSpillFrequency Method for configuring periodic spilling in unit tests. If set, will
- *                           spill every `frequency` records.
- */
-case class Sort(
-    sortOrder: Seq[SortOrder],
-    global: Boolean,
-    child: SparkPlan,
-    testSpillFrequency: Int = 0)
-  extends UnaryNode with CodegenSupport {
-
-  override def output: Seq[Attribute] = child.output
-
-  override def outputOrdering: Seq[SortOrder] = sortOrder
-
-  override def requiredChildDistribution: Seq[Distribution] =
-    if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil
-
-  private val enableRadixSort = sqlContext.conf.enableRadixSort
-
-  override private[sql] lazy val metrics = Map(
-    "sortTime" -> SQLMetrics.createTimingMetric(sparkContext, "sort time"),
-    "peakMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak memory"),
-    "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size"))
-
-  def createSorter(): UnsafeExternalRowSorter = {
-    val ordering = newOrdering(sortOrder, output)
-
-    // The comparator for comparing prefix
-    val boundSortExpression = BindReferences.bindReference(sortOrder.head, output)
-    val prefixComparator = SortPrefixUtils.getPrefixComparator(boundSortExpression)
-
-    val canUseRadixSort = enableRadixSort && sortOrder.length == 1 &&
-      SortPrefixUtils.canSortFullyWithPrefix(boundSortExpression)
-
-    // The generator for prefix
-    val prefixProjection = UnsafeProjection.create(Seq(SortPrefix(boundSortExpression)))
-    val prefixComputer = new UnsafeExternalRowSorter.PrefixComputer {
-      override def computePrefix(row: InternalRow): Long = {
-        prefixProjection.apply(row).getLong(0)
-      }
-    }
-
-    val pageSize = SparkEnv.get.memoryManager.pageSizeBytes
-    val sorter = new UnsafeExternalRowSorter(
-      schema, ordering, prefixComparator, prefixComputer, pageSize, canUseRadixSort)
-
-    if (testSpillFrequency > 0) {
-      sorter.setTestSpillFrequency(testSpillFrequency)
-    }
-    sorter
-  }
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    val peakMemory = longMetric("peakMemory")
-    val spillSize = longMetric("spillSize")
-    val sortTime = longMetric("sortTime")
-
-    child.execute().mapPartitionsInternal { iter =>
-      val sorter = createSorter()
-
-      val metrics = TaskContext.get().taskMetrics()
-      // Remember spill data size of this task before execute this operator so that we can
-      // figure out how many bytes we spilled for this operator.
-      val spillSizeBefore = metrics.memoryBytesSpilled
-      val beforeSort = System.nanoTime()
-
-      val sortedIterator = sorter.sort(iter.asInstanceOf[Iterator[UnsafeRow]])
-
-      sortTime += (System.nanoTime() - beforeSort) / 1000000
-      peakMemory += sorter.getPeakMemoryUsage
-      spillSize += metrics.memoryBytesSpilled - spillSizeBefore
-      metrics.incPeakExecutionMemory(sorter.getPeakMemoryUsage)
-
-      sortedIterator
-    }
-  }
-
-  override def usedInputs: AttributeSet = AttributeSet(Seq.empty)
-
-  override def inputRDDs(): Seq[RDD[InternalRow]] = {
-    child.asInstanceOf[CodegenSupport].inputRDDs()
-  }
-
-  // Name of sorter variable used in codegen.
-  private var sorterVariable: String = _
-
-  override protected def doProduce(ctx: CodegenContext): String = {
-    val needToSort = ctx.freshName("needToSort")
-    ctx.addMutableState("boolean", needToSort, s"$needToSort = true;")
-
-    // Initialize the class member variables. This includes the instance of the Sorter and
-    // the iterator to return sorted rows.
-    val thisPlan = ctx.addReferenceObj("plan", this)
-    sorterVariable = ctx.freshName("sorter")
-    ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, sorterVariable,
-      s"$sorterVariable = $thisPlan.createSorter();")
-    val metrics = ctx.freshName("metrics")
-    ctx.addMutableState(classOf[TaskMetrics].getName, metrics,
-      s"$metrics = org.apache.spark.TaskContext.get().taskMetrics();")
-    val sortedIterator = ctx.freshName("sortedIter")
-    ctx.addMutableState("scala.collection.Iterator<UnsafeRow>", sortedIterator, "")
-
-    val addToSorter = ctx.freshName("addToSorter")
-    ctx.addNewFunction(addToSorter,
-      s"""
-        | private void $addToSorter() throws java.io.IOException {
-        |   ${child.asInstanceOf[CodegenSupport].produce(ctx, this)}
-        | }
-      """.stripMargin.trim)
-
-    // The child could change `copyResult` to true, but we had already consumed all the rows,
-    // so `copyResult` should be reset to `false`.
-    ctx.copyResult = false
-
-    val outputRow = ctx.freshName("outputRow")
-    val peakMemory = metricTerm(ctx, "peakMemory")
-    val spillSize = metricTerm(ctx, "spillSize")
-    val spillSizeBefore = ctx.freshName("spillSizeBefore")
-    val startTime = ctx.freshName("startTime")
-    val sortTime = metricTerm(ctx, "sortTime")
-    s"""
-       | if ($needToSort) {
-       |   long $spillSizeBefore = $metrics.memoryBytesSpilled();
-       |   long $startTime = System.nanoTime();
-       |   $addToSorter();
-       |   $sortedIterator = $sorterVariable.sort();
-       |   $sortTime.add((System.nanoTime() - $startTime) / 1000000);
-       |   $peakMemory.add($sorterVariable.getPeakMemoryUsage());
-       |   $spillSize.add($metrics.memoryBytesSpilled() - $spillSizeBefore);
-       |   $metrics.incPeakExecutionMemory($sorterVariable.getPeakMemoryUsage());
-       |   $needToSort = false;
-       | }
-       |
-       | while ($sortedIterator.hasNext()) {
-       |   UnsafeRow $outputRow = (UnsafeRow)$sortedIterator.next();
-       |   ${consume(ctx, null, outputRow)}
-       |   if (shouldStop()) return;
-       | }
-     """.stripMargin.trim
-  }
-
-  override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
-    s"""
-       |${row.code}
-       |$sorterVariable.insertRow((UnsafeRow)${row.value});
-     """.stripMargin
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
new file mode 100644
index 0000000..0e4d6d7
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import org.apache.spark.{SparkEnv, TaskContext}
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, GenerateUnsafeProjection}
+import org.apache.spark.sql.catalyst.plans.physical.{Distribution, OrderedDistribution, UnspecifiedDistribution}
+import org.apache.spark.sql.execution.metric.SQLMetrics
+import org.apache.spark.sql.types._
+import org.apache.spark.util.collection.unsafe.sort.RadixSort;
+
+/**
+ * Performs (external) sorting.
+ *
+ * @param global when true performs a global sort of all partitions by shuffling the data first
+ *               if necessary.
+ * @param testSpillFrequency Method for configuring periodic spilling in unit tests. If set, will
+ *                           spill every `frequency` records.
+ */
+case class SortExec(
+    sortOrder: Seq[SortOrder],
+    global: Boolean,
+    child: SparkPlan,
+    testSpillFrequency: Int = 0)
+  extends UnaryExecNode with CodegenSupport {
+
+  override def output: Seq[Attribute] = child.output
+
+  override def outputOrdering: Seq[SortOrder] = sortOrder
+
+  override def requiredChildDistribution: Seq[Distribution] =
+    if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil
+
+  private val enableRadixSort = sqlContext.conf.enableRadixSort
+
+  override private[sql] lazy val metrics = Map(
+    "sortTime" -> SQLMetrics.createTimingMetric(sparkContext, "sort time"),
+    "peakMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak memory"),
+    "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size"))
+
+  def createSorter(): UnsafeExternalRowSorter = {
+    val ordering = newOrdering(sortOrder, output)
+
+    // The comparator for comparing prefix
+    val boundSortExpression = BindReferences.bindReference(sortOrder.head, output)
+    val prefixComparator = SortPrefixUtils.getPrefixComparator(boundSortExpression)
+
+    val canUseRadixSort = enableRadixSort && sortOrder.length == 1 &&
+      SortPrefixUtils.canSortFullyWithPrefix(boundSortExpression)
+
+    // The generator for prefix
+    val prefixProjection = UnsafeProjection.create(Seq(SortPrefix(boundSortExpression)))
+    val prefixComputer = new UnsafeExternalRowSorter.PrefixComputer {
+      override def computePrefix(row: InternalRow): Long = {
+        prefixProjection.apply(row).getLong(0)
+      }
+    }
+
+    val pageSize = SparkEnv.get.memoryManager.pageSizeBytes
+    val sorter = new UnsafeExternalRowSorter(
+      schema, ordering, prefixComparator, prefixComputer, pageSize, canUseRadixSort)
+
+    if (testSpillFrequency > 0) {
+      sorter.setTestSpillFrequency(testSpillFrequency)
+    }
+    sorter
+  }
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    val peakMemory = longMetric("peakMemory")
+    val spillSize = longMetric("spillSize")
+    val sortTime = longMetric("sortTime")
+
+    child.execute().mapPartitionsInternal { iter =>
+      val sorter = createSorter()
+
+      val metrics = TaskContext.get().taskMetrics()
+      // Remember spill data size of this task before execute this operator so that we can
+      // figure out how many bytes we spilled for this operator.
+      val spillSizeBefore = metrics.memoryBytesSpilled
+      val beforeSort = System.nanoTime()
+
+      val sortedIterator = sorter.sort(iter.asInstanceOf[Iterator[UnsafeRow]])
+
+      sortTime += (System.nanoTime() - beforeSort) / 1000000
+      peakMemory += sorter.getPeakMemoryUsage
+      spillSize += metrics.memoryBytesSpilled - spillSizeBefore
+      metrics.incPeakExecutionMemory(sorter.getPeakMemoryUsage)
+
+      sortedIterator
+    }
+  }
+
+  override def usedInputs: AttributeSet = AttributeSet(Seq.empty)
+
+  override def inputRDDs(): Seq[RDD[InternalRow]] = {
+    child.asInstanceOf[CodegenSupport].inputRDDs()
+  }
+
+  // Name of sorter variable used in codegen.
+  private var sorterVariable: String = _
+
+  override protected def doProduce(ctx: CodegenContext): String = {
+    val needToSort = ctx.freshName("needToSort")
+    ctx.addMutableState("boolean", needToSort, s"$needToSort = true;")
+
+    // Initialize the class member variables. This includes the instance of the Sorter and
+    // the iterator to return sorted rows.
+    val thisPlan = ctx.addReferenceObj("plan", this)
+    sorterVariable = ctx.freshName("sorter")
+    ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, sorterVariable,
+      s"$sorterVariable = $thisPlan.createSorter();")
+    val metrics = ctx.freshName("metrics")
+    ctx.addMutableState(classOf[TaskMetrics].getName, metrics,
+      s"$metrics = org.apache.spark.TaskContext.get().taskMetrics();")
+    val sortedIterator = ctx.freshName("sortedIter")
+    ctx.addMutableState("scala.collection.Iterator<UnsafeRow>", sortedIterator, "")
+
+    val addToSorter = ctx.freshName("addToSorter")
+    ctx.addNewFunction(addToSorter,
+      s"""
+        | private void $addToSorter() throws java.io.IOException {
+        |   ${child.asInstanceOf[CodegenSupport].produce(ctx, this)}
+        | }
+      """.stripMargin.trim)
+
+    // The child could change `copyResult` to true, but we had already consumed all the rows,
+    // so `copyResult` should be reset to `false`.
+    ctx.copyResult = false
+
+    val outputRow = ctx.freshName("outputRow")
+    val peakMemory = metricTerm(ctx, "peakMemory")
+    val spillSize = metricTerm(ctx, "spillSize")
+    val spillSizeBefore = ctx.freshName("spillSizeBefore")
+    val startTime = ctx.freshName("startTime")
+    val sortTime = metricTerm(ctx, "sortTime")
+    s"""
+       | if ($needToSort) {
+       |   long $spillSizeBefore = $metrics.memoryBytesSpilled();
+       |   long $startTime = System.nanoTime();
+       |   $addToSorter();
+       |   $sortedIterator = $sorterVariable.sort();
+       |   $sortTime.add((System.nanoTime() - $startTime) / 1000000);
+       |   $peakMemory.add($sorterVariable.getPeakMemoryUsage());
+       |   $spillSize.add($metrics.memoryBytesSpilled() - $spillSizeBefore);
+       |   $metrics.incPeakExecutionMemory($sorterVariable.getPeakMemoryUsage());
+       |   $needToSort = false;
+       | }
+       |
+       | while ($sortedIterator.hasNext()) {
+       |   UnsafeRow $outputRow = (UnsafeRow)$sortedIterator.next();
+       |   ${consume(ctx, null, outputRow)}
+       |   if (shouldStop()) return;
+       | }
+     """.stripMargin.trim
+  }
+
+  override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
+    s"""
+       |${row.code}
+       |$sorterVariable.insertRow((UnsafeRow)${row.value});
+     """.stripMargin
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
----------------------------------------------------------------------
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 64d89f2..e28e456 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
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala
----------------------------------------------------------------------
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 247f55d..cb4b1cf 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) =>

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala
----------------------------------------------------------------------
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 8d05ae4..0afa4c7 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))
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d7d0cad0/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
----------------------------------------------------------------------
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 ed6b846..3ce5f28 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
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org