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 2015/01/28 01:08:42 UTC

[1/5] spark git commit: [SPARK-5097][SQL] DataFrame

Repository: spark
Updated Branches:
  refs/heads/master b1b35ca2e -> 119f45d61


http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala
deleted file mode 100644
index 272c0d4..0000000
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala
+++ /dev/null
@@ -1,65 +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.sql.QueryTest
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans._
-
-/* Implicit conversions */
-import org.apache.spark.sql.test.TestSQLContext._
-
-/**
- * This is an example TGF that uses UnresolvedAttributes 'name and 'age to access specific columns
- * from the input data.  These will be replaced during analysis with specific AttributeReferences
- * and then bound to specific ordinals during query planning. While TGFs could also access specific
- * columns using hand-coded ordinals, doing so violates data independence.
- *
- * Note: this is only a rough example of how TGFs can be expressed, the final version will likely
- * involve a lot more sugar for cleaner use in Scala/Java/etc.
- */
-case class ExampleTGF(input: Seq[Expression] = Seq('name, 'age)) extends Generator {
-  def children = input
-  protected def makeOutput() = 'nameAndAge.string :: Nil
-
-  val Seq(nameAttr, ageAttr) = input
-
-  override def eval(input: Row): TraversableOnce[Row] = {
-    val name = nameAttr.eval(input)
-    val age = ageAttr.eval(input).asInstanceOf[Int]
-
-    Iterator(
-      new GenericRow(Array[Any](s"$name is $age years old")),
-      new GenericRow(Array[Any](s"Next year, $name will be ${age + 1} years old")))
-  }
-}
-
-class TgfSuite extends QueryTest {
-  val inputData =
-    logical.LocalRelation('name.string, 'age.int).loadData(
-      ("michael", 29) :: Nil
-    )
-
-  test("simple tgf example") {
-    checkAnswer(
-      inputData.generate(ExampleTGF()),
-      Seq(
-        Row("michael is 29 years old"),
-        Row("Next year, michael will be 30 years old")))
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
index 94d14ac..ef198f8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
@@ -21,11 +21,12 @@ import java.sql.{Date, Timestamp}
 
 import org.apache.spark.sql.TestData._
 import org.apache.spark.sql.catalyst.util._
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.json.JsonRDD.{compatibleType, enforceCorrectType}
 import org.apache.spark.sql.test.TestSQLContext
 import org.apache.spark.sql.test.TestSQLContext._
 import org.apache.spark.sql.types._
-import org.apache.spark.sql.{QueryTest, Row, SQLConf}
+import org.apache.spark.sql.{Literal, QueryTest, Row, SQLConf}
 
 class JsonSuite extends QueryTest {
   import org.apache.spark.sql.json.TestJsonData._
@@ -463,8 +464,8 @@ class JsonSuite extends QueryTest {
     // in the Project.
     checkAnswer(
       jsonSchemaRDD.
-        where('num_str > BigDecimal("92233720368547758060")).
-        select('num_str + 1.2 as Symbol("num")),
+        where('num_str > Literal(BigDecimal("92233720368547758060"))).
+        select(('num_str + Literal(1.2)).as("num")),
       Row(new java.math.BigDecimal("92233720368547758061.2"))
     )
 
@@ -820,7 +821,7 @@ class JsonSuite extends QueryTest {
 
     val schemaRDD1 = applySchema(rowRDD1, schema1)
     schemaRDD1.registerTempTable("applySchema1")
-    val schemaRDD2 = schemaRDD1.toSchemaRDD
+    val schemaRDD2 = schemaRDD1.toDF
     val result = schemaRDD2.toJSON.collect()
     assert(result(0) == "{\"f1\":1,\"f2\":\"A1\",\"f3\":true,\"f4\":[\"1\",\" A1\",\" true\",\" null\"]}")
     assert(result(3) == "{\"f1\":4,\"f2\":\"D4\",\"f3\":true,\"f4\":[\"4\",\" D4\",\" true\",\" 2147483644\"],\"f5\":2147483644}")
@@ -841,7 +842,7 @@ class JsonSuite extends QueryTest {
 
     val schemaRDD3 = applySchema(rowRDD2, schema2)
     schemaRDD3.registerTempTable("applySchema2")
-    val schemaRDD4 = schemaRDD3.toSchemaRDD
+    val schemaRDD4 = schemaRDD3.toDF
     val result2 = schemaRDD4.toJSON.collect()
 
     assert(result2(1) == "{\"f1\":{\"f11\":2,\"f12\":false},\"f2\":{\"B2\":null}}")

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala
index 1e7d3e0..c9bc559 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala
@@ -23,7 +23,7 @@ import parquet.filter2.predicate.{FilterPredicate, Operators}
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal, Predicate, Row}
 import org.apache.spark.sql.test.TestSQLContext
-import org.apache.spark.sql.{QueryTest, SQLConf, SchemaRDD}
+import org.apache.spark.sql.{DataFrame, QueryTest, SQLConf}
 
 /**
  * A test suite that tests Parquet filter2 API based filter pushdown optimization.
@@ -41,15 +41,17 @@ class ParquetFilterSuite extends QueryTest with ParquetTest {
   val sqlContext = TestSQLContext
 
   private def checkFilterPredicate(
-      rdd: SchemaRDD,
+      rdd: DataFrame,
       predicate: Predicate,
       filterClass: Class[_ <: FilterPredicate],
-      checker: (SchemaRDD, Seq[Row]) => Unit,
+      checker: (DataFrame, Seq[Row]) => Unit,
       expected: Seq[Row]): Unit = {
     val output = predicate.collect { case a: Attribute => a }.distinct
 
     withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED -> "true") {
-      val query = rdd.select(output: _*).where(predicate)
+      val query = rdd
+        .select(output.map(e => new org.apache.spark.sql.Column(e)): _*)
+        .where(new org.apache.spark.sql.Column(predicate))
 
       val maybeAnalyzedPredicate = query.queryExecution.executedPlan.collect {
         case plan: ParquetTableScan => plan.columnPruningPred
@@ -71,13 +73,13 @@ class ParquetFilterSuite extends QueryTest with ParquetTest {
 
   private def checkFilterPredicate
       (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Seq[Row])
-      (implicit rdd: SchemaRDD): Unit = {
+      (implicit rdd: DataFrame): Unit = {
     checkFilterPredicate(rdd, predicate, filterClass, checkAnswer(_, _: Seq[Row]), expected)
   }
 
   private def checkFilterPredicate[T]
       (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: T)
-      (implicit rdd: SchemaRDD): Unit = {
+      (implicit rdd: DataFrame): Unit = {
     checkFilterPredicate(predicate, filterClass, Seq(Row(expected)))(rdd)
   }
 
@@ -93,24 +95,24 @@ class ParquetFilterSuite extends QueryTest with ParquetTest {
 
   test("filter pushdown - integer") {
     withParquetRDD((1 to 4).map(i => Tuple1(Option(i)))) { implicit rdd =>
-      checkFilterPredicate('_1.isNull,    classOf[Eq   [_]], Seq.empty[Row])
+      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
       checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
 
-      checkFilterPredicate('_1 === 1, classOf[Eq   [_]], 1)
+      checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1)
       checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
 
-      checkFilterPredicate('_1 < 2,  classOf[Lt  [_]], 1)
-      checkFilterPredicate('_1 > 3,  classOf[Gt  [_]], 4)
+      checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1)
+      checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4)
       checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1)
       checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4)
 
       checkFilterPredicate(Literal(1) === '_1, classOf[Eq  [_]], 1)
-      checkFilterPredicate(Literal(2) >   '_1, classOf[Lt  [_]], 1)
-      checkFilterPredicate(Literal(3) <   '_1, classOf[Gt  [_]], 4)
-      checkFilterPredicate(Literal(1) >=  '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4) <=  '_1, classOf[GtEq[_]], 4)
+      checkFilterPredicate(Literal(2) > '_1, classOf[Lt  [_]], 1)
+      checkFilterPredicate(Literal(3) < '_1, classOf[Gt  [_]], 4)
+      checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1)
+      checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4)
 
-      checkFilterPredicate(!('_1 < 4),         classOf[GtEq[_]],       4)
+      checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4)
       checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3)
       checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or],  Seq(Row(1), Row(4)))
     }
@@ -118,24 +120,24 @@ class ParquetFilterSuite extends QueryTest with ParquetTest {
 
   test("filter pushdown - long") {
     withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit rdd =>
-      checkFilterPredicate('_1.isNull,    classOf[Eq   [_]], Seq.empty[Row])
+      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
       checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
 
-      checkFilterPredicate('_1 === 1, classOf[Eq[_]],    1)
+      checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1)
       checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
 
-      checkFilterPredicate('_1 <  2, classOf[Lt  [_]], 1)
-      checkFilterPredicate('_1 >  3, classOf[Gt  [_]], 4)
+      checkFilterPredicate('_1 <  2, classOf[Lt[_]], 1)
+      checkFilterPredicate('_1 >  3, classOf[Gt[_]], 4)
       checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1)
       checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4)
 
-      checkFilterPredicate(Literal(1) === '_1, classOf[Eq  [_]], 1)
-      checkFilterPredicate(Literal(2) >   '_1, classOf[Lt  [_]], 1)
-      checkFilterPredicate(Literal(3) <   '_1, classOf[Gt  [_]], 4)
+      checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1)
+      checkFilterPredicate(Literal(2) >   '_1, classOf[Lt[_]], 1)
+      checkFilterPredicate(Literal(3) <   '_1, classOf[Gt[_]], 4)
       checkFilterPredicate(Literal(1) >=  '_1, classOf[LtEq[_]], 1)
       checkFilterPredicate(Literal(4) <=  '_1, classOf[GtEq[_]], 4)
 
-      checkFilterPredicate(!('_1 < 4),         classOf[GtEq[_]],       4)
+      checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4)
       checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3)
       checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or],  Seq(Row(1), Row(4)))
     }
@@ -143,24 +145,24 @@ class ParquetFilterSuite extends QueryTest with ParquetTest {
 
   test("filter pushdown - float") {
     withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit rdd =>
-      checkFilterPredicate('_1.isNull,    classOf[Eq   [_]], Seq.empty[Row])
+      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
       checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
 
-      checkFilterPredicate('_1 === 1, classOf[Eq   [_]], 1)
+      checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1)
       checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
 
-      checkFilterPredicate('_1 <  2, classOf[Lt  [_]], 1)
-      checkFilterPredicate('_1 >  3, classOf[Gt  [_]], 4)
+      checkFilterPredicate('_1 <  2, classOf[Lt[_]], 1)
+      checkFilterPredicate('_1 >  3, classOf[Gt[_]], 4)
       checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1)
       checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4)
 
-      checkFilterPredicate(Literal(1) === '_1, classOf[Eq  [_]], 1)
-      checkFilterPredicate(Literal(2) >   '_1, classOf[Lt  [_]], 1)
-      checkFilterPredicate(Literal(3) <   '_1, classOf[Gt  [_]], 4)
-      checkFilterPredicate(Literal(1) >=  '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4) <=  '_1, classOf[GtEq[_]], 4)
+      checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1)
+      checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1)
+      checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4)
+      checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1)
+      checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4)
 
-      checkFilterPredicate(!('_1 < 4),         classOf[GtEq[_]],       4)
+      checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4)
       checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3)
       checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or],  Seq(Row(1), Row(4)))
     }
@@ -168,24 +170,24 @@ class ParquetFilterSuite extends QueryTest with ParquetTest {
 
   test("filter pushdown - double") {
     withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit rdd =>
-      checkFilterPredicate('_1.isNull,    classOf[Eq[_]],    Seq.empty[Row])
+      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
       checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
 
-      checkFilterPredicate('_1 === 1, classOf[Eq   [_]], 1)
+      checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1)
       checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
 
-      checkFilterPredicate('_1 <  2, classOf[Lt  [_]], 1)
-      checkFilterPredicate('_1 >  3, classOf[Gt  [_]], 4)
+      checkFilterPredicate('_1 <  2, classOf[Lt[_]], 1)
+      checkFilterPredicate('_1 >  3, classOf[Gt[_]], 4)
       checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1)
       checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4)
 
       checkFilterPredicate(Literal(1) === '_1, classOf[Eq  [_]], 1)
-      checkFilterPredicate(Literal(2) >   '_1, classOf[Lt  [_]], 1)
-      checkFilterPredicate(Literal(3) <   '_1, classOf[Gt  [_]], 4)
-      checkFilterPredicate(Literal(1) >=  '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4) <=  '_1, classOf[GtEq[_]], 4)
+      checkFilterPredicate(Literal(2) > '_1, classOf[Lt  [_]], 1)
+      checkFilterPredicate(Literal(3) < '_1, classOf[Gt  [_]], 4)
+      checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1)
+      checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4)
 
-      checkFilterPredicate(!('_1 < 4),         classOf[GtEq[_]],       4)
+      checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4)
       checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3)
       checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or],  Seq(Row(1), Row(4)))
     }
@@ -197,30 +199,30 @@ class ParquetFilterSuite extends QueryTest with ParquetTest {
       checkFilterPredicate(
         '_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(i => Row.apply(i.toString)))
 
-      checkFilterPredicate('_1 === "1", classOf[Eq   [_]], "1")
+      checkFilterPredicate('_1 === "1", classOf[Eq[_]], "1")
       checkFilterPredicate('_1 !== "1", classOf[NotEq[_]], (2 to 4).map(i => Row.apply(i.toString)))
 
-      checkFilterPredicate('_1 <  "2", classOf[Lt  [_]], "1")
-      checkFilterPredicate('_1 >  "3", classOf[Gt  [_]], "4")
+      checkFilterPredicate('_1 <  "2", classOf[Lt[_]], "1")
+      checkFilterPredicate('_1 >  "3", classOf[Gt[_]], "4")
       checkFilterPredicate('_1 <= "1", classOf[LtEq[_]], "1")
       checkFilterPredicate('_1 >= "4", classOf[GtEq[_]], "4")
 
-      checkFilterPredicate(Literal("1") === '_1, classOf[Eq  [_]], "1")
-      checkFilterPredicate(Literal("2") >   '_1, classOf[Lt  [_]], "1")
-      checkFilterPredicate(Literal("3") <   '_1, classOf[Gt  [_]], "4")
-      checkFilterPredicate(Literal("1") >=  '_1, classOf[LtEq[_]], "1")
-      checkFilterPredicate(Literal("4") <=  '_1, classOf[GtEq[_]], "4")
+      checkFilterPredicate(Literal("1") === '_1, classOf[Eq[_]], "1")
+      checkFilterPredicate(Literal("2") > '_1, classOf[Lt[_]], "1")
+      checkFilterPredicate(Literal("3") < '_1, classOf[Gt[_]], "4")
+      checkFilterPredicate(Literal("1") >= '_1, classOf[LtEq[_]], "1")
+      checkFilterPredicate(Literal("4") <= '_1, classOf[GtEq[_]], "4")
 
-      checkFilterPredicate(!('_1 < "4"),           classOf[GtEq[_]],       "4")
+      checkFilterPredicate(!('_1 < "4"), classOf[GtEq[_]], "4")
       checkFilterPredicate('_1 > "2" && '_1 < "4", classOf[Operators.And], "3")
-      checkFilterPredicate('_1 < "2" || '_1 > "3", classOf[Operators.Or],  Seq(Row("1"), Row("4")))
+      checkFilterPredicate('_1 < "2" || '_1 > "3", classOf[Operators.Or], Seq(Row("1"), Row("4")))
     }
   }
 
   def checkBinaryFilterPredicate
       (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Seq[Row])
-      (implicit rdd: SchemaRDD): Unit = {
-    def checkBinaryAnswer(rdd: SchemaRDD, expected: Seq[Row]) = {
+      (implicit rdd: DataFrame): Unit = {
+    def checkBinaryAnswer(rdd: DataFrame, expected: Seq[Row]) = {
       assertResult(expected.map(_.getAs[Array[Byte]](0).mkString(",")).toSeq.sorted) {
         rdd.map(_.getAs[Array[Byte]](0).mkString(",")).collect().toSeq.sorted
       }
@@ -231,7 +233,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest {
 
   def checkBinaryFilterPredicate
       (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Array[Byte])
-      (implicit rdd: SchemaRDD): Unit = {
+      (implicit rdd: DataFrame): Unit = {
     checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(rdd)
   }
 
@@ -249,16 +251,16 @@ class ParquetFilterSuite extends QueryTest with ParquetTest {
       checkBinaryFilterPredicate(
         '_1 !== 1.b, classOf[NotEq[_]], (2 to 4).map(i => Row.apply(i.b)).toSeq)
 
-      checkBinaryFilterPredicate('_1 <  2.b, classOf[Lt  [_]], 1.b)
-      checkBinaryFilterPredicate('_1 >  3.b, classOf[Gt  [_]], 4.b)
+      checkBinaryFilterPredicate('_1 < 2.b, classOf[Lt[_]], 1.b)
+      checkBinaryFilterPredicate('_1 > 3.b, classOf[Gt[_]], 4.b)
       checkBinaryFilterPredicate('_1 <= 1.b, classOf[LtEq[_]], 1.b)
       checkBinaryFilterPredicate('_1 >= 4.b, classOf[GtEq[_]], 4.b)
 
-      checkBinaryFilterPredicate(Literal(1.b) === '_1, classOf[Eq  [_]], 1.b)
-      checkBinaryFilterPredicate(Literal(2.b) >   '_1, classOf[Lt  [_]], 1.b)
-      checkBinaryFilterPredicate(Literal(3.b) <   '_1, classOf[Gt  [_]], 4.b)
-      checkBinaryFilterPredicate(Literal(1.b) >=  '_1, classOf[LtEq[_]], 1.b)
-      checkBinaryFilterPredicate(Literal(4.b) <=  '_1, classOf[GtEq[_]], 4.b)
+      checkBinaryFilterPredicate(Literal(1.b) === '_1, classOf[Eq[_]], 1.b)
+      checkBinaryFilterPredicate(Literal(2.b) > '_1, classOf[Lt[_]], 1.b)
+      checkBinaryFilterPredicate(Literal(3.b) < '_1, classOf[Gt[_]], 4.b)
+      checkBinaryFilterPredicate(Literal(1.b) >= '_1, classOf[LtEq[_]], 1.b)
+      checkBinaryFilterPredicate(Literal(4.b) <= '_1, classOf[GtEq[_]], 4.b)
 
       checkBinaryFilterPredicate(!('_1 < 4.b), classOf[GtEq[_]], 4.b)
       checkBinaryFilterPredicate('_1 > 2.b && '_1 < 4.b, classOf[Operators.And], 3.b)

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala
index a57e4e8..f03b3a3 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala
@@ -32,12 +32,13 @@ import parquet.schema.{MessageType, MessageTypeParser}
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.spark.sql.{DataFrame, QueryTest, SQLConf}
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.expressions.Row
 import org.apache.spark.sql.test.TestSQLContext
 import org.apache.spark.sql.test.TestSQLContext._
 import org.apache.spark.sql.types.DecimalType
-import org.apache.spark.sql.{QueryTest, SQLConf, SchemaRDD}
 
 // Write support class for nested groups: ParquetWriter initializes GroupWriteSupport
 // with an empty configuration (it is after all not intended to be used in this way?)
@@ -97,11 +98,11 @@ class ParquetIOSuite extends QueryTest with ParquetTest {
   }
 
   test("fixed-length decimals") {
-    def makeDecimalRDD(decimal: DecimalType): SchemaRDD =
+    def makeDecimalRDD(decimal: DecimalType): DataFrame =
       sparkContext
         .parallelize(0 to 1000)
         .map(i => Tuple1(i / 100.0))
-        .select('_1 cast decimal)
+        .select($"_1" cast decimal as "abcd")
 
     for ((precision, scale) <- Seq((5, 2), (1, 0), (1, 1), (18, 10), (18, 17))) {
       withTempPath { dir =>

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/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 7900b3e..a33cf11 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
@@ -17,6 +17,8 @@
 
 package org.apache.spark.sql.sources
 
+import scala.language.existentials
+
 import org.apache.spark.sql._
 import org.apache.spark.sql.types._
 

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
----------------------------------------------------------------------
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
index 7385952..bb19ac2 100755
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
@@ -23,6 +23,7 @@ import java.io._
 import java.util.{ArrayList => JArrayList}
 
 import jline.{ConsoleReader, History}
+
 import org.apache.commons.lang.StringUtils
 import org.apache.commons.logging.LogFactory
 import org.apache.hadoop.conf.Configuration
@@ -39,7 +40,6 @@ import org.apache.thrift.transport.TSocket
 
 import org.apache.spark.Logging
 import org.apache.spark.sql.hive.HiveShim
-import org.apache.spark.sql.hive.thriftserver.HiveThriftServerShim
 
 private[hive] object SparkSQLCLIDriver {
   private var prompt = "spark-sql"

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala
----------------------------------------------------------------------
diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala
index 166c56b..ea9d61d 100644
--- a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala
+++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala
@@ -32,7 +32,7 @@ import org.apache.hive.service.cli.operation.ExecuteStatementOperation
 import org.apache.hive.service.cli.session.HiveSession
 
 import org.apache.spark.Logging
-import org.apache.spark.sql.{SQLConf, SchemaRDD, Row => SparkRow}
+import org.apache.spark.sql.{DataFrame, SQLConf, Row => SparkRow}
 import org.apache.spark.sql.execution.SetCommand
 import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
 import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes}
@@ -71,7 +71,7 @@ private[hive] class SparkExecuteStatementOperation(
     sessionToActivePool: SMap[SessionHandle, String])
   extends ExecuteStatementOperation(parentSession, statement, confOverlay) with Logging {
 
-  private var result: SchemaRDD = _
+  private var result: DataFrame = _
   private var iter: Iterator[SparkRow] = _
   private var dataTypes: Array[DataType] = _
 
@@ -202,7 +202,7 @@ private[hive] class SparkExecuteStatementOperation(
         val useIncrementalCollect =
           hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean
         if (useIncrementalCollect) {
-          result.toLocalIterator
+          result.rdd.toLocalIterator
         } else {
           result.collect().iterator
         }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala
----------------------------------------------------------------------
diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala
index eaf7a1d..71e3954 100644
--- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala
+++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala
@@ -30,7 +30,7 @@ import org.apache.hive.service.cli.operation.ExecuteStatementOperation
 import org.apache.hive.service.cli.session.HiveSession
 
 import org.apache.spark.Logging
-import org.apache.spark.sql.{Row => SparkRow, SQLConf, SchemaRDD}
+import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLConf}
 import org.apache.spark.sql.execution.SetCommand
 import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
 import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes}
@@ -72,7 +72,7 @@ private[hive] class SparkExecuteStatementOperation(
   // NOTE: `runInBackground` is set to `false` intentionally to disable asynchronous execution
   extends ExecuteStatementOperation(parentSession, statement, confOverlay, false) with Logging {
 
-  private var result: SchemaRDD = _
+  private var result: DataFrame = _
   private var iter: Iterator[SparkRow] = _
   private var dataTypes: Array[DataType] = _
 
@@ -173,7 +173,7 @@ private[hive] class SparkExecuteStatementOperation(
         val useIncrementalCollect =
           hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean
         if (useIncrementalCollect) {
-          result.toLocalIterator
+          result.rdd.toLocalIterator
         } else {
           result.collect().iterator
         }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index 9d2cfd8..b746942 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -64,15 +64,15 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
     getConf("spark.sql.hive.convertMetastoreParquet", "true") == "true"
 
   override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution =
-    new this.QueryExecution { val logical = plan }
+    new this.QueryExecution(plan)
 
-  override def sql(sqlText: String): SchemaRDD = {
+  override def sql(sqlText: String): DataFrame = {
     val substituted = new VariableSubstitution().substitute(hiveconf, sqlText)
     // TODO: Create a framework for registering parsers instead of just hardcoding if statements.
     if (conf.dialect == "sql") {
       super.sql(substituted)
     } else if (conf.dialect == "hiveql") {
-      new SchemaRDD(this, ddlParser(sqlText, false).getOrElse(HiveQl.parseSql(substituted)))
+      new DataFrame(this, ddlParser(sqlText, false).getOrElse(HiveQl.parseSql(substituted)))
     }  else {
       sys.error(s"Unsupported SQL dialect: ${conf.dialect}.  Try 'sql' or 'hiveql'")
     }
@@ -352,7 +352,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
   override protected[sql] val planner = hivePlanner
 
   /** Extends QueryExecution with hive specific features. */
-  protected[sql] abstract class QueryExecution extends super.QueryExecution {
+  protected[sql] class QueryExecution(logicalPlan: LogicalPlan)
+    extends super.QueryExecution(logicalPlan) {
 
     /**
      * Returns the result as a hive compatible sequence of strings.  For native commands, the

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/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 6952b12..ace9329 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
@@ -20,7 +20,7 @@ package org.apache.spark.sql.hive
 import scala.collection.JavaConversions._
 
 import org.apache.spark.annotation.Experimental
-import org.apache.spark.sql.{SQLContext, SchemaRDD, Strategy}
+import org.apache.spark.sql.{Column, DataFrame, SQLContext, Strategy}
 import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
@@ -55,16 +55,15 @@ private[hive] trait HiveStrategies {
    */
   @Experimental
   object ParquetConversion extends Strategy {
-    implicit class LogicalPlanHacks(s: SchemaRDD) {
-      def lowerCase =
-        new SchemaRDD(s.sqlContext, s.logicalPlan)
+    implicit class LogicalPlanHacks(s: DataFrame) {
+      def lowerCase = new DataFrame(s.sqlContext, s.logicalPlan)
 
       def addPartitioningAttributes(attrs: Seq[Attribute]) = {
         // Don't add the partitioning key if its already present in the data.
         if (attrs.map(_.name).toSet.subsetOf(s.logicalPlan.output.map(_.name).toSet)) {
           s
         } else {
-          new SchemaRDD(
+          new DataFrame(
             s.sqlContext,
             s.logicalPlan transform {
               case p: ParquetRelation => p.copy(partitioningAttributes = attrs)
@@ -97,13 +96,13 @@ private[hive] trait HiveStrategies {
         // We are going to throw the predicates and projection back at the whole optimization
         // sequence so lets unresolve all the attributes, allowing them to be rebound to the
         // matching parquet attributes.
-        val unresolvedOtherPredicates = otherPredicates.map(_ transform {
+        val unresolvedOtherPredicates = new Column(otherPredicates.map(_ transform {
           case a: AttributeReference => UnresolvedAttribute(a.name)
-        }).reduceOption(And).getOrElse(Literal(true))
+        }).reduceOption(And).getOrElse(Literal(true)))
 
-        val unresolvedProjection = projectList.map(_ transform {
+        val unresolvedProjection: Seq[Column] = projectList.map(_ transform {
           case a: AttributeReference => UnresolvedAttribute(a.name)
-        })
+        }).map(new Column(_))
 
         try {
           if (relation.hiveQlTable.isPartitioned) {

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
index 47431ce..8e70ae8 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
@@ -99,7 +99,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
   override def runSqlHive(sql: String): Seq[String] = super.runSqlHive(rewritePaths(sql))
 
   override def executePlan(plan: LogicalPlan): this.QueryExecution =
-    new this.QueryExecution { val logical = plan }
+    new this.QueryExecution(plan)
 
   /** Fewer partitions to speed up testing. */
   protected[sql] override lazy val conf: SQLConf = new SQLConf {
@@ -150,8 +150,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
 
   val describedTable = "DESCRIBE (\\w+)".r
 
-  protected[hive] class HiveQLQueryExecution(hql: String) extends this.QueryExecution {
-    lazy val logical = HiveQl.parseSql(hql)
+  protected[hive] class HiveQLQueryExecution(hql: String)
+    extends this.QueryExecution(HiveQl.parseSql(hql)) {
     def hiveExec() = runSqlHive(hql)
     override def toString = hql + "\n" + super.toString
   }
@@ -159,7 +159,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
   /**
    * Override QueryExecution with special debug workflow.
    */
-  abstract class QueryExecution extends super.QueryExecution {
+  class QueryExecution(logicalPlan: LogicalPlan)
+    extends super.QueryExecution(logicalPlan) {
     override lazy val analyzed = {
       val describedTables = logical match {
         case HiveNativeCommand(describedTable(tbl)) => tbl :: Nil

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala
index f320d73..ba39129 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala
@@ -36,12 +36,12 @@ class QueryTest extends PlanTest {
   /**
    * Runs the plan and makes sure the answer contains all of the keywords, or the
    * none of keywords are listed in the answer
-   * @param rdd the [[SchemaRDD]] to be executed
+   * @param rdd the [[DataFrame]] to be executed
    * @param exists true for make sure the keywords are listed in the output, otherwise
    *               to make sure none of the keyword are not listed in the output
    * @param keywords keyword in string array
    */
-  def checkExistence(rdd: SchemaRDD, exists: Boolean, keywords: String*) {
+  def checkExistence(rdd: DataFrame, exists: Boolean, keywords: String*) {
     val outputs = rdd.collect().map(_.mkString).mkString
     for (key <- keywords) {
       if (exists) {
@@ -54,10 +54,10 @@ class QueryTest extends PlanTest {
 
   /**
    * Runs the plan and makes sure the answer matches the expected result.
-   * @param rdd the [[SchemaRDD]] to be executed
+   * @param rdd the [[DataFrame]] to be executed
    * @param expectedAnswer the expected result, can either be an Any, Seq[Product], or Seq[ Seq[Any] ].
    */
-  protected def checkAnswer(rdd: SchemaRDD, expectedAnswer: Seq[Row]): Unit = {
+  protected def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Unit = {
     val isSorted = rdd.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty
     def prepareAnswer(answer: Seq[Row]): Seq[Row] = {
       // Converts data to types that we can do equality comparison using Scala collections.
@@ -101,7 +101,7 @@ class QueryTest extends PlanTest {
     }
   }
 
-  protected def checkAnswer(rdd: SchemaRDD, expectedAnswer: Row): Unit = {
+  protected def checkAnswer(rdd: DataFrame, expectedAnswer: Row): Unit = {
     checkAnswer(rdd, Seq(expectedAnswer))
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/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 f95a6b4..61e5117 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 org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation}
 import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.hive.test.TestHive._
-import org.apache.spark.sql.{QueryTest, SchemaRDD}
+import org.apache.spark.sql.{DataFrame, QueryTest}
 import org.apache.spark.storage.RDDBlockId
 
 class CachedTableSuite extends QueryTest {
@@ -28,7 +28,7 @@ class CachedTableSuite extends QueryTest {
    * Throws a test failed exception when the number of cached tables differs from the expected
    * number.
    */
-  def assertCached(query: SchemaRDD, numCachedTables: Int = 1): Unit = {
+  def assertCached(query: DataFrame, numCachedTables: Int = 1): Unit = {
     val planWithCaching = query.queryExecution.withCachedData
     val cachedData = planWithCaching collect {
       case cached: InMemoryRelation => cached

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
index 0e6636d..5775d83 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
@@ -52,7 +52,7 @@ class InsertIntoHiveTableSuite extends QueryTest {
     // Make sure the table has been updated.
     checkAnswer(
       sql("SELECT * FROM createAndInsertTest"),
-      testData.toSchemaRDD.collect().toSeq ++ testData.toSchemaRDD.collect().toSeq
+      testData.toDF.collect().toSeq ++ testData.toDF.collect().toSeq
     )
 
     // Now overwrite.

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/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 df72be7..d67b00b 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
@@ -27,11 +27,12 @@ import scala.util.Try
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars
 
 import org.apache.spark.{SparkFiles, SparkException}
+import org.apache.spark.sql.{DataFrame, Row}
 import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.hive._
 import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.hive.test.TestHive._
-import org.apache.spark.sql.{SQLConf, Row, SchemaRDD}
 
 case class TestData(a: Int, b: String)
 
@@ -473,7 +474,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
     }
   }
 
-  def isExplanation(result: SchemaRDD) = {
+  def isExplanation(result: DataFrame) = {
     val explanation = result.select('plan).collect().map { case Row(plan: String) => plan }
     explanation.contains("== Physical Plan ==")
   }
@@ -842,7 +843,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
     val testVal = "test.val.0"
     val nonexistentKey = "nonexistent"
     val KV = "([^=]+)=([^=]*)".r
-    def collectResults(rdd: SchemaRDD): Set[(String, String)] =
+    def collectResults(rdd: DataFrame): Set[(String, String)] =
       rdd.collect().map {
         case Row(key: String, value: String) => key -> value
         case Row(KV(key, value)) => key -> value

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
index 16f77a4..a081227 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
@@ -17,9 +17,10 @@
 
 package org.apache.spark.sql.hive.execution
 
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.hive.test.TestHive._
-import org.apache.spark.sql.Row
 
 import org.apache.spark.util.Utils
 
@@ -82,10 +83,10 @@ class HiveTableScanSuite extends HiveComparisonTest {
     sql("create table spark_4959 (col1 string)")
     sql("""insert into table spark_4959 select "hi" from src limit 1""")
     table("spark_4959").select(
-      'col1.as('CaseSensitiveColName),
-      'col1.as('CaseSensitiveColName2)).registerTempTable("spark_4959_2")
+      'col1.as("CaseSensitiveColName"),
+      'col1.as("CaseSensitiveColName2")).registerTempTable("spark_4959_2")
 
-    assert(sql("select CaseSensitiveColName from spark_4959_2").first() === Row("hi"))
-    assert(sql("select casesensitivecolname from spark_4959_2").first() === Row("hi"))
+    assert(sql("select CaseSensitiveColName from spark_4959_2").head() === Row("hi"))
+    assert(sql("select casesensitivecolname from spark_4959_2").head() === Row("hi"))
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
index f2374a2..dd0df1a 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
@@ -58,7 +58,7 @@ class HiveUdfSuite extends QueryTest {
         |       getStruct(1).f3,
         |       getStruct(1).f4,
         |       getStruct(1).f5 FROM src LIMIT 1
-      """.stripMargin).first() === Row(1, 2, 3, 4, 5))
+      """.stripMargin).head() === Row(1, 2, 3, 4, 5))
   }
   
   test("SPARK-4785 When called with arguments referring column fields, PMOD throws NPE") {


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


[2/5] spark git commit: [SPARK-5097][SQL] DataFrame

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/dsl/package.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/dsl/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/dsl/package.scala
new file mode 100644
index 0000000..29c3d26
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/dsl/package.scala
@@ -0,0 +1,495 @@
+/*
+ * 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
+
+import java.sql.{Timestamp, Date}
+
+import scala.language.implicitConversions
+import scala.reflect.runtime.universe.{TypeTag, typeTag}
+
+import org.apache.spark.sql.catalyst.ScalaReflection
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.types.DataType
+
+
+package object dsl {
+
+  implicit def symbolToColumn(s: Symbol): ColumnName = new ColumnName(s.name)
+
+  /** Converts $"col name" into an [[Column]]. */
+  implicit class StringToColumn(val sc: StringContext) extends AnyVal {
+    def $(args: Any*): ColumnName = {
+      new ColumnName(sc.s(args :_*))
+    }
+  }
+
+  private[this] implicit def toColumn(expr: Expression): Column = new Column(expr)
+
+  def sum(e: Column): Column = Sum(e.expr)
+  def sumDistinct(e: Column): Column = SumDistinct(e.expr)
+  def count(e: Column): Column = Count(e.expr)
+
+  @scala.annotation.varargs
+  def countDistinct(expr: Column, exprs: Column*): Column =
+    CountDistinct((expr +: exprs).map(_.expr))
+
+  def avg(e: Column): Column = Average(e.expr)
+  def first(e: Column): Column = First(e.expr)
+  def last(e: Column): Column = Last(e.expr)
+  def min(e: Column): Column = Min(e.expr)
+  def max(e: Column): Column = Max(e.expr)
+  def upper(e: Column): Column = Upper(e.expr)
+  def lower(e: Column): Column = Lower(e.expr)
+  def sqrt(e: Column): Column = Sqrt(e.expr)
+  def abs(e: Column): Column = Abs(e.expr)
+
+  // scalastyle:off
+
+  object literals {
+
+    implicit def booleanToLiteral(b: Boolean): Column = Literal(b)
+
+    implicit def byteToLiteral(b: Byte): Column = Literal(b)
+
+    implicit def shortToLiteral(s: Short): Column = Literal(s)
+
+    implicit def intToLiteral(i: Int): Column = Literal(i)
+
+    implicit def longToLiteral(l: Long): Column = Literal(l)
+
+    implicit def floatToLiteral(f: Float): Column = Literal(f)
+
+    implicit def doubleToLiteral(d: Double): Column = Literal(d)
+
+    implicit def stringToLiteral(s: String): Column = Literal(s)
+
+    implicit def dateToLiteral(d: Date): Column = Literal(d)
+
+    implicit def bigDecimalToLiteral(d: BigDecimal): Column = Literal(d.underlying())
+
+    implicit def bigDecimalToLiteral(d: java.math.BigDecimal): Column = Literal(d)
+
+    implicit def timestampToLiteral(t: Timestamp): Column = Literal(t)
+
+    implicit def binaryToLiteral(a: Array[Byte]): Column = Literal(a)
+  }
+
+
+  /* Use the following code to generate:
+  (0 to 22).map { x =>
+    val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"})
+    val typeTags = (1 to x).map(i => s"A$i: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _)
+    val args = (1 to x).map(i => s"arg$i: Column").mkString(", ")
+    val argsInUdf = (1 to x).map(i => s"arg$i.expr").mkString(", ")
+    println(s"""
+    /**
+     * Call a Scala function of ${x} arguments as user-defined function (UDF), and automatically
+     * infer the data types based on the function's signature.
+     */
+    def callUDF[$typeTags](f: Function$x[$types]${if (args.length > 0) ", " + args else ""}): Column = {
+      ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq($argsInUdf))
+    }""")
+  }
+
+  (0 to 22).map { x =>
+    val args = (1 to x).map(i => s"arg$i: Column").mkString(", ")
+    val fTypes = Seq.fill(x + 1)("_").mkString(", ")
+    val argsInUdf = (1 to x).map(i => s"arg$i.expr").mkString(", ")
+    println(s"""
+    /**
+     * Call a Scala function of ${x} arguments as user-defined function (UDF). This requires
+     * you to specify the return data type.
+     */
+    def callUDF(f: Function$x[$fTypes], returnType: DataType${if (args.length > 0) ", " + args else ""}): Column = {
+      ScalaUdf(f, returnType, Seq($argsInUdf))
+    }""")
+  }
+  }
+  */
+  /**
+   * Call a Scala function of 0 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag](f: Function0[RT]): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq())
+  }
+
+  /**
+   * Call a Scala function of 1 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT], arg1: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr))
+  }
+
+  /**
+   * Call a Scala function of 2 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT], arg1: Column, arg2: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr))
+  }
+
+  /**
+   * Call a Scala function of 3 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT], arg1: Column, arg2: Column, arg3: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr))
+  }
+
+  /**
+   * Call a Scala function of 4 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](f: Function4[A1, A2, A3, A4, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr))
+  }
+
+  /**
+   * Call a Scala function of 5 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](f: Function5[A1, A2, A3, A4, A5, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr))
+  }
+
+  /**
+   * Call a Scala function of 6 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](f: Function6[A1, A2, A3, A4, A5, A6, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr))
+  }
+
+  /**
+   * Call a Scala function of 7 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](f: Function7[A1, A2, A3, A4, A5, A6, A7, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr))
+  }
+
+  /**
+   * Call a Scala function of 8 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](f: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr))
+  }
+
+  /**
+   * Call a Scala function of 9 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](f: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr))
+  }
+
+  /**
+   * Call a Scala function of 10 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](f: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr))
+  }
+
+  /**
+   * Call a Scala function of 11 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](f: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr))
+  }
+
+  /**
+   * Call a Scala function of 12 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](f: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr))
+  }
+
+  /**
+   * Call a Scala function of 13 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](f: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr))
+  }
+
+  /**
+   * Call a Scala function of 14 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](f: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr))
+  }
+
+  /**
+   * Call a Scala function of 15 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](f: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr))
+  }
+
+  /**
+   * Call a Scala function of 16 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](f: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr))
+  }
+
+  /**
+   * Call a Scala function of 17 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](f: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr))
+  }
+
+  /**
+   * Call a Scala function of 18 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](f: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr))
+  }
+
+  /**
+   * Call a Scala function of 19 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](f: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr))
+  }
+
+  /**
+   * Call a Scala function of 20 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](f: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr))
+  }
+
+  /**
+   * Call a Scala function of 21 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](f: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column, arg21: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr, arg21.expr))
+  }
+
+  /**
+   * Call a Scala function of 22 arguments as user-defined function (UDF), and automatically
+   * infer the data types based on the function's signature.
+   */
+  def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](f: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column, arg21: Column, arg22: Column): Column = {
+    ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr, arg21.expr, arg22.expr))
+  }
+
+  //////////////////////////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * Call a Scala function of 0 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function0[_], returnType: DataType): Column = {
+    ScalaUdf(f, returnType, Seq())
+  }
+
+  /**
+   * Call a Scala function of 1 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function1[_, _], returnType: DataType, arg1: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr))
+  }
+
+  /**
+   * Call a Scala function of 2 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function2[_, _, _], returnType: DataType, arg1: Column, arg2: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr))
+  }
+
+  /**
+   * Call a Scala function of 3 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function3[_, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr))
+  }
+
+  /**
+   * Call a Scala function of 4 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function4[_, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr))
+  }
+
+  /**
+   * Call a Scala function of 5 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function5[_, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr))
+  }
+
+  /**
+   * Call a Scala function of 6 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function6[_, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr))
+  }
+
+  /**
+   * Call a Scala function of 7 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function7[_, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr))
+  }
+
+  /**
+   * Call a Scala function of 8 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function8[_, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr))
+  }
+
+  /**
+   * Call a Scala function of 9 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function9[_, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr))
+  }
+
+  /**
+   * Call a Scala function of 10 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr))
+  }
+
+  /**
+   * Call a Scala function of 11 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr))
+  }
+
+  /**
+   * Call a Scala function of 12 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr))
+  }
+
+  /**
+   * Call a Scala function of 13 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr))
+  }
+
+  /**
+   * Call a Scala function of 14 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr))
+  }
+
+  /**
+   * Call a Scala function of 15 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr))
+  }
+
+  /**
+   * Call a Scala function of 16 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr))
+  }
+
+  /**
+   * Call a Scala function of 17 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr))
+  }
+
+  /**
+   * Call a Scala function of 18 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr))
+  }
+
+  /**
+   * Call a Scala function of 19 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr))
+  }
+
+  /**
+   * Call a Scala function of 20 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr))
+  }
+
+  /**
+   * Call a Scala function of 21 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column, arg21: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr, arg21.expr))
+  }
+
+  /**
+   * Call a Scala function of 22 arguments as user-defined function (UDF). This requires
+   * you to specify the return data type.
+   */
+  def callUDF(f: Function22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column, arg21: Column, arg22: Column): Column = {
+    ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr, arg21.expr, arg22.expr))
+  }
+
+  // scalastyle:on
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
index 52a31f0..6fba76c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.execution
 import org.apache.spark.Logging
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{SchemaRDD, SQLConf, SQLContext}
+import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext}
 import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.expressions.{Row, Attribute}
 import org.apache.spark.sql.catalyst.plans.logical
@@ -137,7 +137,9 @@ case class CacheTableCommand(
     isLazy: Boolean) extends RunnableCommand {
 
   override def run(sqlContext: SQLContext) = {
-    plan.foreach(p => new SchemaRDD(sqlContext, p).registerTempTable(tableName))
+    plan.foreach { logicalPlan =>
+      sqlContext.registerRDDAsTable(new DataFrame(sqlContext, logicalPlan), tableName)
+    }
     sqlContext.cacheTable(tableName)
 
     if (!isLazy) {
@@ -159,7 +161,7 @@ case class CacheTableCommand(
 case class UncacheTableCommand(tableName: String) extends RunnableCommand {
 
   override def run(sqlContext: SQLContext) = {
-    sqlContext.table(tableName).unpersist()
+    sqlContext.table(tableName).unpersist(blocking = false)
     Seq.empty[Row]
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/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 4d7e338..aeb0960 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
@@ -22,7 +22,7 @@ import scala.collection.mutable.HashSet
 import org.apache.spark.{AccumulatorParam, Accumulator, SparkContext}
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.SparkContext._
-import org.apache.spark.sql.{SchemaRDD, Row}
+import org.apache.spark.sql.{DataFrame, Row}
 import org.apache.spark.sql.catalyst.trees.TreeNodeRef
 import org.apache.spark.sql.types._
 
@@ -42,7 +42,7 @@ package object debug {
    * Augments SchemaRDDs with debug methods.
    */
   @DeveloperApi
-  implicit class DebugQuery(query: SchemaRDD) {
+  implicit class DebugQuery(query: DataFrame) {
     def debug(): Unit = {
       val plan = query.queryExecution.executedPlan
       val visited = new collection.mutable.HashSet[TreeNodeRef]()

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/package.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala
index 6dd39be..7c49b52 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala
@@ -37,5 +37,5 @@ package object sql {
    * Converts a logical plan into zero or more SparkPlans.
    */
   @DeveloperApi
-  type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan]
+  protected[sql] type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan]
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala
index 02ce1b3..0b312ef 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala
@@ -23,7 +23,7 @@ import scala.reflect.ClassTag
 import scala.reflect.runtime.universe.TypeTag
 import scala.util.Try
 
-import org.apache.spark.sql.{SQLContext, SchemaRDD}
+import org.apache.spark.sql.{DataFrame, SQLContext}
 import org.apache.spark.sql.catalyst.util
 import org.apache.spark.util.Utils
 
@@ -100,7 +100,7 @@ trait ParquetTest {
    */
   protected def withParquetRDD[T <: Product: ClassTag: TypeTag]
       (data: Seq[T])
-      (f: SchemaRDD => Unit): Unit = {
+      (f: DataFrame => Unit): Unit = {
     withParquetFile(data)(path => f(parquetFile(path)))
   }
 
@@ -120,7 +120,7 @@ trait ParquetTest {
       (data: Seq[T], tableName: String)
       (f: => Unit): Unit = {
     withParquetRDD(data) { rdd =>
-      rdd.registerTempTable(tableName)
+      sqlContext.registerRDDAsTable(rdd, tableName)
       withTempTable(tableName)(f)
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala
index 37853d4..d13f2ce 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala
@@ -18,19 +18,18 @@
 package org.apache.spark.sql.sources
 
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.Row
-import org.apache.spark.sql._
+import org.apache.spark.sql.{Row, Strategy}
 import org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, AttributeSet, Expression, NamedExpression}
 import org.apache.spark.sql.catalyst.planning.PhysicalOperation
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution
 
 /**
  * A Strategy for planning scans over data sources defined using the sources API.
  */
 private[sql] object DataSourceStrategy extends Strategy {
-  def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
+  def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match {
     case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: CatalystScan)) =>
       pruneFilterProjectRaw(
         l,
@@ -112,23 +111,26 @@ private[sql] object DataSourceStrategy extends Strategy {
     }
   }
 
+  /** Turn Catalyst [[Expression]]s into data source [[Filter]]s. */
   protected[sql] def selectFilters(filters: Seq[Expression]): Seq[Filter] = filters.collect {
-    case expressions.EqualTo(a: Attribute, Literal(v, _)) => EqualTo(a.name, v)
-    case expressions.EqualTo(Literal(v, _), a: Attribute) => EqualTo(a.name, v)
+    case expressions.EqualTo(a: Attribute, expressions.Literal(v, _)) => EqualTo(a.name, v)
+    case expressions.EqualTo(expressions.Literal(v, _), a: Attribute) => EqualTo(a.name, v)
 
-    case expressions.GreaterThan(a: Attribute, Literal(v, _)) => GreaterThan(a.name, v)
-    case expressions.GreaterThan(Literal(v, _), a: Attribute) => LessThan(a.name, v)
+    case expressions.GreaterThan(a: Attribute, expressions.Literal(v, _)) => GreaterThan(a.name, v)
+    case expressions.GreaterThan(expressions.Literal(v, _), a: Attribute) => LessThan(a.name, v)
 
-    case expressions.LessThan(a: Attribute, Literal(v, _)) => LessThan(a.name, v)
-    case expressions.LessThan(Literal(v, _), a: Attribute) => GreaterThan(a.name, v)
+    case expressions.LessThan(a: Attribute, expressions.Literal(v, _)) => LessThan(a.name, v)
+    case expressions.LessThan(expressions.Literal(v, _), a: Attribute) => GreaterThan(a.name, v)
 
-    case expressions.GreaterThanOrEqual(a: Attribute, Literal(v, _)) =>
+    case expressions.GreaterThanOrEqual(a: Attribute, expressions.Literal(v, _)) =>
       GreaterThanOrEqual(a.name, v)
-    case expressions.GreaterThanOrEqual(Literal(v, _), a: Attribute) =>
+    case expressions.GreaterThanOrEqual(expressions.Literal(v, _), a: Attribute) =>
       LessThanOrEqual(a.name, v)
 
-    case expressions.LessThanOrEqual(a: Attribute, Literal(v, _)) => LessThanOrEqual(a.name, v)
-    case expressions.LessThanOrEqual(Literal(v, _), a: Attribute) => GreaterThanOrEqual(a.name, v)
+    case expressions.LessThanOrEqual(a: Attribute, expressions.Literal(v, _)) =>
+      LessThanOrEqual(a.name, v)
+    case expressions.LessThanOrEqual(expressions.Literal(v, _), a: Attribute) =>
+      GreaterThanOrEqual(a.name, v)
 
     case expressions.InSet(a: Attribute, set) => In(a.name, set.toArray)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
index 171b816..b4af91a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.sources
 import scala.language.implicitConversions
 
 import org.apache.spark.Logging
-import org.apache.spark.sql.{SchemaRDD, SQLContext}
+import org.apache.spark.sql.{DataFrame, SQLContext}
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.AbstractSparkSQLParser
 import org.apache.spark.sql.execution.RunnableCommand
@@ -225,7 +225,8 @@ private [sql] case class CreateTempTableUsing(
 
   def run(sqlContext: SQLContext) = {
     val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options)
-    new SchemaRDD(sqlContext, LogicalRelation(resolved.relation)).registerTempTable(tableName)
+    sqlContext.registerRDDAsTable(
+      new DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName)
     Seq.empty
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
index f9c0822..2564c84 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.test
 import scala.language.implicitConversions
 
 import org.apache.spark.{SparkConf, SparkContext}
-import org.apache.spark.sql.{SchemaRDD, SQLConf, SQLContext}
+import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 
 /** A SQLContext that can be used for local testing. */
@@ -40,8 +40,8 @@ object TestSQLContext
    * Turn a logical plan into a SchemaRDD. This should be removed once we have an easier way to
    * construct SchemaRDD directly out of local data without relying on implicits.
    */
-  protected[sql] implicit def logicalPlanToSparkQuery(plan: LogicalPlan): SchemaRDD = {
-    new SchemaRDD(this, plan)
+  protected[sql] implicit def logicalPlanToSparkQuery(plan: LogicalPlan): DataFrame = {
+    new DataFrame(this, plan)
   }
 
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java
----------------------------------------------------------------------
diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java
index 9ff4047..e558893 100644
--- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java
+++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java
@@ -61,7 +61,7 @@ public class JavaAPISuite implements Serializable {
       }
     }, DataTypes.IntegerType);
 
-    Row result = sqlContext.sql("SELECT stringLengthTest('test')").first();
+    Row result = sqlContext.sql("SELECT stringLengthTest('test')").head();
     assert(result.getInt(0) == 4);
   }
 
@@ -81,7 +81,7 @@ public class JavaAPISuite implements Serializable {
       }
     }, DataTypes.IntegerType);
 
-    Row result = sqlContext.sql("SELECT stringLengthTest('test', 'test2')").first();
+    Row result = sqlContext.sql("SELECT stringLengthTest('test', 'test2')").head();
     assert(result.getInt(0) == 9);
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
----------------------------------------------------------------------
diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
index 9e96738..badd00d 100644
--- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
+++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
@@ -98,8 +98,8 @@ public class JavaApplySchemaSuite implements Serializable {
     fields.add(DataTypes.createStructField("age", DataTypes.IntegerType, false));
     StructType schema = DataTypes.createStructType(fields);
 
-    SchemaRDD schemaRDD = javaSqlCtx.applySchema(rowRDD.rdd(), schema);
-    schemaRDD.registerTempTable("people");
+    DataFrame df = javaSqlCtx.applySchema(rowRDD.rdd(), schema);
+    df.registerTempTable("people");
     Row[] actual = javaSqlCtx.sql("SELECT * FROM people").collect();
 
     List<Row> expected = new ArrayList<Row>(2);
@@ -147,17 +147,17 @@ public class JavaApplySchemaSuite implements Serializable {
         null,
         "this is another simple string."));
 
-    SchemaRDD schemaRDD1 = javaSqlCtx.jsonRDD(jsonRDD.rdd());
-    StructType actualSchema1 = schemaRDD1.schema();
+    DataFrame df1 = javaSqlCtx.jsonRDD(jsonRDD.rdd());
+    StructType actualSchema1 = df1.schema();
     Assert.assertEquals(expectedSchema, actualSchema1);
-    schemaRDD1.registerTempTable("jsonTable1");
+    df1.registerTempTable("jsonTable1");
     List<Row> actual1 = javaSqlCtx.sql("select * from jsonTable1").collectAsList();
     Assert.assertEquals(expectedResult, actual1);
 
-    SchemaRDD schemaRDD2 = javaSqlCtx.jsonRDD(jsonRDD.rdd(), expectedSchema);
-    StructType actualSchema2 = schemaRDD2.schema();
+    DataFrame df2 = javaSqlCtx.jsonRDD(jsonRDD.rdd(), expectedSchema);
+    StructType actualSchema2 = df2.schema();
     Assert.assertEquals(expectedSchema, actualSchema2);
-    schemaRDD2.registerTempTable("jsonTable2");
+    df2.registerTempTable("jsonTable2");
     List<Row> actual2 = javaSqlCtx.sql("select * from jsonTable2").collectAsList();
     Assert.assertEquals(expectedResult, actual2);
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/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 cfc037c..3476315 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
@@ -19,6 +19,7 @@ package org.apache.spark.sql
 
 import org.apache.spark.sql.TestData._
 import org.apache.spark.sql.columnar._
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.test.TestSQLContext._
 import org.apache.spark.storage.{StorageLevel, RDDBlockId}
 

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala
index afbfe21..a5848f2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala
@@ -17,12 +17,10 @@
 
 package org.apache.spark.sql
 
-import org.apache.spark.sql.catalyst.analysis._
-import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.types._
 
 /* Implicits */
-import org.apache.spark.sql.catalyst.dsl._
 import org.apache.spark.sql.test.TestSQLContext._
 
 import scala.language.postfixOps
@@ -44,46 +42,46 @@ class DslQuerySuite extends QueryTest {
 
   test("agg") {
     checkAnswer(
-      testData2.groupBy('a)('a, sum('b)),
+      testData2.groupBy("a").agg($"a", sum($"b")),
       Seq(Row(1,3), Row(2,3), Row(3,3))
     )
     checkAnswer(
-      testData2.groupBy('a)('a, sum('b) as 'totB).aggregate(sum('totB)),
+      testData2.groupBy("a").agg($"a", sum($"b").as("totB")).agg(sum('totB)),
       Row(9)
     )
     checkAnswer(
-      testData2.aggregate(sum('b)),
+      testData2.agg(sum('b)),
       Row(9)
     )
   }
 
   test("convert $\"attribute name\" into unresolved attribute") {
     checkAnswer(
-      testData.where($"key" === 1).select($"value"),
+      testData.where($"key" === Literal(1)).select($"value"),
       Row("1"))
   }
 
   test("convert Scala Symbol 'attrname into unresolved attribute") {
     checkAnswer(
-      testData.where('key === 1).select('value),
+      testData.where('key === Literal(1)).select('value),
       Row("1"))
   }
 
   test("select *") {
     checkAnswer(
-      testData.select(Star(None)),
+      testData.select($"*"),
       testData.collect().toSeq)
   }
 
   test("simple select") {
     checkAnswer(
-      testData.where('key === 1).select('value),
+      testData.where('key === Literal(1)).select('value),
       Row("1"))
   }
 
   test("select with functions") {
     checkAnswer(
-      testData.select(sum('value), avg('value), count(1)),
+      testData.select(sum('value), avg('value), count(Literal(1))),
       Row(5050.0, 50.5, 100))
 
     checkAnswer(
@@ -120,46 +118,19 @@ class DslQuerySuite extends QueryTest {
 
     checkAnswer(
       arrayData.orderBy('data.getItem(0).asc),
-      arrayData.toSchemaRDD.collect().sortBy(_.getAs[Seq[Int]](0)(0)).toSeq)
+      arrayData.toDF.collect().sortBy(_.getAs[Seq[Int]](0)(0)).toSeq)
 
     checkAnswer(
       arrayData.orderBy('data.getItem(0).desc),
-      arrayData.toSchemaRDD.collect().sortBy(_.getAs[Seq[Int]](0)(0)).reverse.toSeq)
+      arrayData.toDF.collect().sortBy(_.getAs[Seq[Int]](0)(0)).reverse.toSeq)
 
     checkAnswer(
       arrayData.orderBy('data.getItem(1).asc),
-      arrayData.toSchemaRDD.collect().sortBy(_.getAs[Seq[Int]](0)(1)).toSeq)
+      arrayData.toDF.collect().sortBy(_.getAs[Seq[Int]](0)(1)).toSeq)
 
     checkAnswer(
       arrayData.orderBy('data.getItem(1).desc),
-      arrayData.toSchemaRDD.collect().sortBy(_.getAs[Seq[Int]](0)(1)).reverse.toSeq)
-  }
-
-  test("partition wide sorting") {
-    // 2 partitions totally, and
-    // Partition #1 with values:
-    //    (1, 1)
-    //    (1, 2)
-    //    (2, 1)
-    // Partition #2 with values:
-    //    (2, 2)
-    //    (3, 1)
-    //    (3, 2)
-    checkAnswer(
-      testData2.sortBy('a.asc, 'b.asc),
-      Seq(Row(1,1), Row(1,2), Row(2,1), Row(2,2), Row(3,1), Row(3,2)))
-
-    checkAnswer(
-      testData2.sortBy('a.asc, 'b.desc),
-      Seq(Row(1,2), Row(1,1), Row(2,1), Row(2,2), Row(3,2), Row(3,1)))
-
-    checkAnswer(
-      testData2.sortBy('a.desc, 'b.desc),
-      Seq(Row(2,1), Row(1,2), Row(1,1), Row(3,2), Row(3,1), Row(2,2)))
-
-    checkAnswer(
-      testData2.sortBy('a.desc, 'b.asc),
-      Seq(Row(2,1), Row(1,1), Row(1,2), Row(3,1), Row(3,2), Row(2,2)))
+      arrayData.toDF.collect().sortBy(_.getAs[Seq[Int]](0)(1)).reverse.toSeq)
   }
 
   test("limit") {
@@ -176,71 +147,51 @@ class DslQuerySuite extends QueryTest {
       mapData.take(1).map(r => Row.fromSeq(r.productIterator.toSeq)))
   }
 
-  test("SPARK-3395 limit distinct") {
-    val filtered = TestData.testData2
-      .distinct()
-      .orderBy(SortOrder('a, Ascending), SortOrder('b, Ascending))
-      .limit(1)
-      .registerTempTable("onerow")
-    checkAnswer(
-      sql("select * from onerow inner join testData2 on onerow.a = testData2.a"),
-      Row(1, 1, 1, 1) ::
-      Row(1, 1, 1, 2) :: Nil)
-  }
-
-  test("SPARK-3858 generator qualifiers are discarded") {
-    checkAnswer(
-      arrayData.as('ad)
-        .generate(Explode("data" :: Nil, 'data), alias = Some("ex"))
-        .select("ex.data".attr),
-      Seq(1, 2, 3, 2, 3, 4).map(Row(_)))
-  }
-
   test("average") {
     checkAnswer(
-      testData2.aggregate(avg('a)),
+      testData2.agg(avg('a)),
       Row(2.0))
 
     checkAnswer(
-      testData2.aggregate(avg('a), sumDistinct('a)), // non-partial
+      testData2.agg(avg('a), sumDistinct('a)), // non-partial
       Row(2.0, 6.0) :: Nil)
 
     checkAnswer(
-      decimalData.aggregate(avg('a)),
+      decimalData.agg(avg('a)),
       Row(new java.math.BigDecimal(2.0)))
     checkAnswer(
-      decimalData.aggregate(avg('a), sumDistinct('a)), // non-partial
+      decimalData.agg(avg('a), sumDistinct('a)), // non-partial
       Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(6)) :: Nil)
 
     checkAnswer(
-      decimalData.aggregate(avg('a cast DecimalType(10, 2))),
+      decimalData.agg(avg('a cast DecimalType(10, 2))),
       Row(new java.math.BigDecimal(2.0)))
     checkAnswer(
-      decimalData.aggregate(avg('a cast DecimalType(10, 2)), sumDistinct('a cast DecimalType(10, 2))), // non-partial
+      decimalData.agg(avg('a cast DecimalType(10, 2)), sumDistinct('a cast DecimalType(10, 2))), // non-partial
       Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(6)) :: Nil)
   }
 
   test("null average") {
     checkAnswer(
-      testData3.aggregate(avg('b)),
+      testData3.agg(avg('b)),
       Row(2.0))
 
     checkAnswer(
-      testData3.aggregate(avg('b), countDistinct('b)),
+      testData3.agg(avg('b), countDistinct('b)),
       Row(2.0, 1))
 
     checkAnswer(
-      testData3.aggregate(avg('b), sumDistinct('b)), // non-partial
+      testData3.agg(avg('b), sumDistinct('b)), // non-partial
       Row(2.0, 2.0))
   }
 
   test("zero average") {
     checkAnswer(
-      emptyTableData.aggregate(avg('a)),
+      emptyTableData.agg(avg('a)),
       Row(null))
 
     checkAnswer(
-      emptyTableData.aggregate(avg('a), sumDistinct('b)), // non-partial
+      emptyTableData.agg(avg('a), sumDistinct('b)), // non-partial
       Row(null, null))
   }
 
@@ -248,28 +199,28 @@ class DslQuerySuite extends QueryTest {
     assert(testData2.count() === testData2.map(_ => 1).count())
 
     checkAnswer(
-      testData2.aggregate(count('a), sumDistinct('a)), // non-partial
+      testData2.agg(count('a), sumDistinct('a)), // non-partial
       Row(6, 6.0))
   }
 
   test("null count") {
     checkAnswer(
-      testData3.groupBy('a)('a, count('b)),
+      testData3.groupBy('a).agg('a, count('b)),
       Seq(Row(1,0), Row(2, 1))
     )
 
     checkAnswer(
-      testData3.groupBy('a)('a, count('a + 'b)),
+      testData3.groupBy('a).agg('a, count('a + 'b)),
       Seq(Row(1,0), Row(2, 1))
     )
 
     checkAnswer(
-      testData3.aggregate(count('a), count('b), count(1), countDistinct('a), countDistinct('b)),
+      testData3.agg(count('a), count('b), count(Literal(1)), countDistinct('a), countDistinct('b)),
       Row(2, 1, 2, 2, 1)
     )
 
     checkAnswer(
-      testData3.aggregate(count('b), countDistinct('b), sumDistinct('b)), // non-partial
+      testData3.agg(count('b), countDistinct('b), sumDistinct('b)), // non-partial
       Row(1, 1, 2)
     )
   }
@@ -278,19 +229,19 @@ class DslQuerySuite extends QueryTest {
     assert(emptyTableData.count() === 0)
 
     checkAnswer(
-      emptyTableData.aggregate(count('a), sumDistinct('a)), // non-partial
+      emptyTableData.agg(count('a), sumDistinct('a)), // non-partial
       Row(0, null))
   }
 
   test("zero sum") {
     checkAnswer(
-      emptyTableData.aggregate(sum('a)),
+      emptyTableData.agg(sum('a)),
       Row(null))
   }
 
   test("zero sum distinct") {
     checkAnswer(
-      emptyTableData.aggregate(sumDistinct('a)),
+      emptyTableData.agg(sumDistinct('a)),
       Row(null))
   }
 
@@ -320,7 +271,7 @@ class DslQuerySuite extends QueryTest {
 
     checkAnswer(
       // SELECT *, foo(key, value) FROM testData
-      testData.select(Star(None), foo.call('key, 'value)).limit(3),
+      testData.select($"*", callUDF(foo, 'key, 'value)).limit(3),
       Row(1, "1", "11") :: Row(2, "2", "22") :: Row(3, "3", "33") :: Nil
     )
   }
@@ -362,7 +313,7 @@ class DslQuerySuite extends QueryTest {
   test("upper") {
     checkAnswer(
       lowerCaseData.select(upper('l)),
-      ('a' to 'd').map(c => Row(c.toString.toUpperCase()))
+      ('a' to 'd').map(c => Row(c.toString.toUpperCase))
     )
 
     checkAnswer(
@@ -379,7 +330,7 @@ class DslQuerySuite extends QueryTest {
   test("lower") {
     checkAnswer(
       upperCaseData.select(lower('L)),
-      ('A' to 'F').map(c => Row(c.toString.toLowerCase()))
+      ('A' to 'F').map(c => Row(c.toString.toLowerCase))
     )
 
     checkAnswer(

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/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 cd36da7..7971372 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
@@ -20,19 +20,20 @@ package org.apache.spark.sql
 import org.scalatest.BeforeAndAfterEach
 
 import org.apache.spark.sql.TestData._
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
-import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, LeftOuter, RightOuter}
 import org.apache.spark.sql.execution.joins._
 import org.apache.spark.sql.test.TestSQLContext._
 
+
 class JoinSuite extends QueryTest with BeforeAndAfterEach {
   // Ensures tables are loaded.
   TestData
 
   test("equi-join is hash-join") {
-    val x = testData2.as('x)
-    val y = testData2.as('y)
-    val join = x.join(y, Inner, Some("x.a".attr === "y.a".attr)).queryExecution.analyzed
+    val x = testData2.as("x")
+    val y = testData2.as("y")
+    val join = x.join(y, $"x.a" === $"y.a", "inner").queryExecution.analyzed
     val planned = planner.HashJoin(join)
     assert(planned.size === 1)
   }
@@ -105,17 +106,16 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
   }
 
   test("multiple-key equi-join is hash-join") {
-    val x = testData2.as('x)
-    val y = testData2.as('y)
-    val join = x.join(y, Inner,
-      Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)).queryExecution.analyzed
+    val x = testData2.as("x")
+    val y = testData2.as("y")
+    val join = x.join(y, ($"x.a" === $"y.a") && ($"x.b" === $"y.b")).queryExecution.analyzed
     val planned = planner.HashJoin(join)
     assert(planned.size === 1)
   }
 
   test("inner join where, one match per row") {
     checkAnswer(
-      upperCaseData.join(lowerCaseData, Inner).where('n === 'N),
+      upperCaseData.join(lowerCaseData).where('n === 'N),
       Seq(
         Row(1, "A", 1, "a"),
         Row(2, "B", 2, "b"),
@@ -126,7 +126,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
 
   test("inner join ON, one match per row") {
     checkAnswer(
-      upperCaseData.join(lowerCaseData, Inner, Some('n === 'N)),
+      upperCaseData.join(lowerCaseData, $"n" === $"N"),
       Seq(
         Row(1, "A", 1, "a"),
         Row(2, "B", 2, "b"),
@@ -136,10 +136,10 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
   }
 
   test("inner join, where, multiple matches") {
-    val x = testData2.where('a === 1).as('x)
-    val y = testData2.where('a === 1).as('y)
+    val x = testData2.where($"a" === Literal(1)).as("x")
+    val y = testData2.where($"a" === Literal(1)).as("y")
     checkAnswer(
-      x.join(y).where("x.a".attr === "y.a".attr),
+      x.join(y).where($"x.a" === $"y.a"),
       Row(1,1,1,1) ::
       Row(1,1,1,2) ::
       Row(1,2,1,1) ::
@@ -148,22 +148,21 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
   }
 
   test("inner join, no matches") {
-    val x = testData2.where('a === 1).as('x)
-    val y = testData2.where('a === 2).as('y)
+    val x = testData2.where($"a" === Literal(1)).as("x")
+    val y = testData2.where($"a" === Literal(2)).as("y")
     checkAnswer(
-      x.join(y).where("x.a".attr === "y.a".attr),
+      x.join(y).where($"x.a" === $"y.a"),
       Nil)
   }
 
   test("big inner join, 4 matches per row") {
     val bigData = testData.unionAll(testData).unionAll(testData).unionAll(testData)
-    val bigDataX = bigData.as('x)
-    val bigDataY = bigData.as('y)
+    val bigDataX = bigData.as("x")
+    val bigDataY = bigData.as("y")
 
     checkAnswer(
-      bigDataX.join(bigDataY).where("x.key".attr === "y.key".attr),
-      testData.flatMap(
-        row => Seq.fill(16)(Row.merge(row, row))).collect().toSeq)
+      bigDataX.join(bigDataY).where($"x.key" === $"y.key"),
+      testData.rdd.flatMap(row => Seq.fill(16)(Row.merge(row, row))).collect().toSeq)
   }
 
   test("cartisian product join") {
@@ -177,7 +176,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
 
   test("left outer join") {
     checkAnswer(
-      upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N)),
+      upperCaseData.join(lowerCaseData, $"n" === $"N", "left"),
       Row(1, "A", 1, "a") ::
         Row(2, "B", 2, "b") ::
         Row(3, "C", 3, "c") ::
@@ -186,7 +185,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
         Row(6, "F", null, null) :: Nil)
 
     checkAnswer(
-      upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'n > 1)),
+      upperCaseData.join(lowerCaseData, $"n" === $"N" && $"n" > Literal(1), "left"),
       Row(1, "A", null, null) ::
         Row(2, "B", 2, "b") ::
         Row(3, "C", 3, "c") ::
@@ -195,7 +194,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
         Row(6, "F", null, null) :: Nil)
 
     checkAnswer(
-      upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'N > 1)),
+      upperCaseData.join(lowerCaseData, $"n" === $"N" && $"N" > Literal(1), "left"),
       Row(1, "A", null, null) ::
         Row(2, "B", 2, "b") ::
         Row(3, "C", 3, "c") ::
@@ -204,7 +203,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
         Row(6, "F", null, null) :: Nil)
 
     checkAnswer(
-      upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'l > 'L)),
+      upperCaseData.join(lowerCaseData, $"n" === $"N" && $"l" > $"L", "left"),
       Row(1, "A", 1, "a") ::
         Row(2, "B", 2, "b") ::
         Row(3, "C", 3, "c") ::
@@ -240,7 +239,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
 
   test("right outer join") {
     checkAnswer(
-      lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N)),
+      lowerCaseData.join(upperCaseData, $"n" === $"N", "right"),
       Row(1, "a", 1, "A") ::
         Row(2, "b", 2, "B") ::
         Row(3, "c", 3, "C") ::
@@ -248,7 +247,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
         Row(null, null, 5, "E") ::
         Row(null, null, 6, "F") :: Nil)
     checkAnswer(
-      lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N && 'n > 1)),
+      lowerCaseData.join(upperCaseData, $"n" === $"N" && $"n" > Literal(1), "right"),
       Row(null, null, 1, "A") ::
         Row(2, "b", 2, "B") ::
         Row(3, "c", 3, "C") ::
@@ -256,7 +255,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
         Row(null, null, 5, "E") ::
         Row(null, null, 6, "F") :: Nil)
     checkAnswer(
-      lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N && 'N > 1)),
+      lowerCaseData.join(upperCaseData, $"n" === $"N" && $"N" > Literal(1), "right"),
       Row(null, null, 1, "A") ::
         Row(2, "b", 2, "B") ::
         Row(3, "c", 3, "C") ::
@@ -264,7 +263,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
         Row(null, null, 5, "E") ::
         Row(null, null, 6, "F") :: Nil)
     checkAnswer(
-      lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N && 'l > 'L)),
+      lowerCaseData.join(upperCaseData, $"n" === $"N" && $"l" > $"L", "right"),
       Row(1, "a", 1, "A") ::
         Row(2, "b", 2, "B") ::
         Row(3, "c", 3, "C") ::
@@ -299,14 +298,14 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
   }
 
   test("full outer join") {
-    upperCaseData.where('N <= 4).registerTempTable("left")
-    upperCaseData.where('N >= 3).registerTempTable("right")
+    upperCaseData.where('N <= Literal(4)).registerTempTable("left")
+    upperCaseData.where('N >= Literal(3)).registerTempTable("right")
 
     val left = UnresolvedRelation(Seq("left"), None)
     val right = UnresolvedRelation(Seq("right"), None)
 
     checkAnswer(
-      left.join(right, FullOuter, Some("left.N".attr === "right.N".attr)),
+      left.join(right, $"left.N" === $"right.N", "full"),
       Row(1, "A", null, null) ::
         Row(2, "B", null, null) ::
         Row(3, "C", 3, "C") ::
@@ -315,7 +314,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
         Row(null, null, 6, "F") :: Nil)
 
     checkAnswer(
-      left.join(right, FullOuter, Some(("left.N".attr === "right.N".attr) && ("left.N".attr !== 3))),
+      left.join(right, ($"left.N" === $"right.N") && ($"left.N" !== Literal(3)), "full"),
       Row(1, "A", null, null) ::
         Row(2, "B", null, null) ::
         Row(3, "C", null, null) ::
@@ -325,7 +324,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
         Row(null, null, 6, "F") :: Nil)
 
     checkAnswer(
-      left.join(right, FullOuter, Some(("left.N".attr === "right.N".attr) && ("right.N".attr !== 3))),
+      left.join(right, ($"left.N" === $"right.N") && ($"right.N" !== Literal(3)), "full"),
       Row(1, "A", null, null) ::
         Row(2, "B", null, null) ::
         Row(3, "C", null, null) ::

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
index 42a21c1..07c52de 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
@@ -26,12 +26,12 @@ class QueryTest extends PlanTest {
   /**
    * Runs the plan and makes sure the answer contains all of the keywords, or the
    * none of keywords are listed in the answer
-   * @param rdd the [[SchemaRDD]] to be executed
+   * @param rdd the [[DataFrame]] to be executed
    * @param exists true for make sure the keywords are listed in the output, otherwise
    *               to make sure none of the keyword are not listed in the output
    * @param keywords keyword in string array
    */
-  def checkExistence(rdd: SchemaRDD, exists: Boolean, keywords: String*) {
+  def checkExistence(rdd: DataFrame, exists: Boolean, keywords: String*) {
     val outputs = rdd.collect().map(_.mkString).mkString
     for (key <- keywords) {
       if (exists) {
@@ -44,10 +44,10 @@ class QueryTest extends PlanTest {
 
   /**
    * Runs the plan and makes sure the answer matches the expected result.
-   * @param rdd the [[SchemaRDD]] to be executed
+   * @param rdd the [[DataFrame]] to be executed
    * @param expectedAnswer the expected result, can either be an Any, Seq[Product], or Seq[ Seq[Any] ].
    */
-  protected def checkAnswer(rdd: SchemaRDD, expectedAnswer: Seq[Row]): Unit = {
+  protected def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Unit = {
     val isSorted = rdd.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty
     def prepareAnswer(answer: Seq[Row]): Seq[Row] = {
       // Converts data to types that we can do equality comparison using Scala collections.
@@ -91,7 +91,7 @@ class QueryTest extends PlanTest {
     }
   }
 
-  protected def checkAnswer(rdd: SchemaRDD, expectedAnswer: Row): Unit = {
+  protected def checkAnswer(rdd: DataFrame, expectedAnswer: Row): Unit = {
     checkAnswer(rdd, Seq(expectedAnswer))
   }
 
@@ -102,7 +102,7 @@ class QueryTest extends PlanTest {
   }
 
   /** Asserts that a given SchemaRDD will be executed using the given number of cached results. */
-  def assertCached(query: SchemaRDD, numCachedTables: Int = 1): Unit = {
+  def assertCached(query: DataFrame, numCachedTables: Int = 1): Unit = {
     val planWithCaching = query.queryExecution.withCachedData
     val cachedData = planWithCaching collect {
       case cached: InMemoryRelation => cached

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/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 03b44ca..4fff99c 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
@@ -21,6 +21,7 @@ import java.util.TimeZone
 
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.types._
@@ -29,6 +30,7 @@ import org.apache.spark.sql.types._
 import org.apache.spark.sql.TestData._
 import org.apache.spark.sql.test.TestSQLContext._
 
+
 class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
   // Make sure the tables are loaded.
   TestData
@@ -381,8 +383,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
   }
 
   test("big inner join, 4 matches per row") {
-
-
     checkAnswer(
       sql(
         """
@@ -396,7 +396,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
           |   SELECT * FROM testData UNION ALL
           |   SELECT * FROM testData) y
           |WHERE x.key = y.key""".stripMargin),
-      testData.flatMap(
+      testData.rdd.flatMap(
         row => Seq.fill(16)(Row.merge(row, row))).collect().toSeq)
   }
 
@@ -742,7 +742,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
   }
 
   test("metadata is propagated correctly") {
-    val person = sql("SELECT * FROM person")
+    val person: DataFrame = sql("SELECT * FROM person")
     val schema = person.schema
     val docKey = "doc"
     val docValue = "first name"
@@ -751,14 +751,14 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
       .build()
     val schemaWithMeta = new StructType(Array(
       schema("id"), schema("name").copy(metadata = metadata), schema("age")))
-    val personWithMeta = applySchema(person, schemaWithMeta)
-    def validateMetadata(rdd: SchemaRDD): Unit = {
+    val personWithMeta = applySchema(person.rdd, schemaWithMeta)
+    def validateMetadata(rdd: DataFrame): Unit = {
       assert(rdd.schema("name").metadata.getString(docKey) == docValue)
     }
     personWithMeta.registerTempTable("personWithMeta")
-    validateMetadata(personWithMeta.select('name))
-    validateMetadata(personWithMeta.select("name".attr))
-    validateMetadata(personWithMeta.select('id, 'name))
+    validateMetadata(personWithMeta.select($"name"))
+    validateMetadata(personWithMeta.select($"name"))
+    validateMetadata(personWithMeta.select($"id", $"name"))
     validateMetadata(sql("SELECT * FROM personWithMeta"))
     validateMetadata(sql("SELECT id, name FROM personWithMeta"))
     validateMetadata(sql("SELECT * FROM personWithMeta JOIN salary ON id = personId"))

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala
index 808ed52..fffa2b7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql
 import java.sql.Timestamp
 
 import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.test._
 
 /* Implicits */
@@ -29,11 +30,11 @@ case class TestData(key: Int, value: String)
 
 object TestData {
   val testData = TestSQLContext.sparkContext.parallelize(
-    (1 to 100).map(i => TestData(i, i.toString))).toSchemaRDD
+    (1 to 100).map(i => TestData(i, i.toString))).toDF
   testData.registerTempTable("testData")
 
   val negativeData = TestSQLContext.sparkContext.parallelize(
-    (1 to 100).map(i => TestData(-i, (-i).toString))).toSchemaRDD
+    (1 to 100).map(i => TestData(-i, (-i).toString))).toDF
   negativeData.registerTempTable("negativeData")
 
   case class LargeAndSmallInts(a: Int, b: Int)
@@ -44,7 +45,7 @@ object TestData {
       LargeAndSmallInts(2147483645, 1) ::
       LargeAndSmallInts(2, 2) ::
       LargeAndSmallInts(2147483646, 1) ::
-      LargeAndSmallInts(3, 2) :: Nil).toSchemaRDD
+      LargeAndSmallInts(3, 2) :: Nil).toDF
   largeAndSmallInts.registerTempTable("largeAndSmallInts")
 
   case class TestData2(a: Int, b: Int)
@@ -55,7 +56,7 @@ object TestData {
       TestData2(2, 1) ::
       TestData2(2, 2) ::
       TestData2(3, 1) ::
-      TestData2(3, 2) :: Nil, 2).toSchemaRDD
+      TestData2(3, 2) :: Nil, 2).toDF
   testData2.registerTempTable("testData2")
 
   case class DecimalData(a: BigDecimal, b: BigDecimal)
@@ -67,7 +68,7 @@ object TestData {
       DecimalData(2, 1) ::
       DecimalData(2, 2) ::
       DecimalData(3, 1) ::
-      DecimalData(3, 2) :: Nil).toSchemaRDD
+      DecimalData(3, 2) :: Nil).toDF
   decimalData.registerTempTable("decimalData")
 
   case class BinaryData(a: Array[Byte], b: Int)
@@ -77,17 +78,17 @@ object TestData {
       BinaryData("22".getBytes(), 5) ::
       BinaryData("122".getBytes(), 3) ::
       BinaryData("121".getBytes(), 2) ::
-      BinaryData("123".getBytes(), 4) :: Nil).toSchemaRDD
+      BinaryData("123".getBytes(), 4) :: Nil).toDF
   binaryData.registerTempTable("binaryData")
 
   case class TestData3(a: Int, b: Option[Int])
   val testData3 =
     TestSQLContext.sparkContext.parallelize(
       TestData3(1, None) ::
-      TestData3(2, Some(2)) :: Nil).toSchemaRDD
+      TestData3(2, Some(2)) :: Nil).toDF
   testData3.registerTempTable("testData3")
 
-  val emptyTableData = logical.LocalRelation('a.int, 'b.int)
+  val emptyTableData = logical.LocalRelation($"a".int, $"b".int)
 
   case class UpperCaseData(N: Int, L: String)
   val upperCaseData =
@@ -97,7 +98,7 @@ object TestData {
       UpperCaseData(3, "C") ::
       UpperCaseData(4, "D") ::
       UpperCaseData(5, "E") ::
-      UpperCaseData(6, "F") :: Nil).toSchemaRDD
+      UpperCaseData(6, "F") :: Nil).toDF
   upperCaseData.registerTempTable("upperCaseData")
 
   case class LowerCaseData(n: Int, l: String)
@@ -106,7 +107,7 @@ object TestData {
       LowerCaseData(1, "a") ::
       LowerCaseData(2, "b") ::
       LowerCaseData(3, "c") ::
-      LowerCaseData(4, "d") :: Nil).toSchemaRDD
+      LowerCaseData(4, "d") :: Nil).toDF
   lowerCaseData.registerTempTable("lowerCaseData")
 
   case class ArrayData(data: Seq[Int], nestedData: Seq[Seq[Int]])
@@ -200,6 +201,6 @@ object TestData {
     TestSQLContext.sparkContext.parallelize(
       ComplexData(Map(1 -> "1"), TestData(1, "1"), Seq(1), true)
         :: ComplexData(Map(2 -> "2"), TestData(2, "2"), Seq(2), false)
-        :: Nil).toSchemaRDD
+        :: Nil).toDF
   complexData.registerTempTable("complexData")
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
index 0c98120..5abd7b9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql
 
+import org.apache.spark.sql.dsl.StringToColumn
 import org.apache.spark.sql.test._
 
 /* Implicits */
@@ -28,17 +29,17 @@ class UDFSuite extends QueryTest {
 
   test("Simple UDF") {
     udf.register("strLenScala", (_: String).length)
-    assert(sql("SELECT strLenScala('test')").first().getInt(0) === 4)
+    assert(sql("SELECT strLenScala('test')").head().getInt(0) === 4)
   }
 
   test("ZeroArgument UDF") {
     udf.register("random0", () => { Math.random()})
-    assert(sql("SELECT random0()").first().getDouble(0) >= 0.0)
+    assert(sql("SELECT random0()").head().getDouble(0) >= 0.0)
   }
 
   test("TwoArgument UDF") {
     udf.register("strLenScala", (_: String).length + (_:Int))
-    assert(sql("SELECT strLenScala('test', 1)").first().getInt(0) === 5)
+    assert(sql("SELECT strLenScala('test', 1)").head().getInt(0) === 5)
   }
 
   test("struct UDF") {
@@ -46,7 +47,7 @@ class UDFSuite extends QueryTest {
 
     val result=
       sql("SELECT returnStruct('test', 'test2') as ret")
-        .select("ret.f1".attr).first().getString(0)
-    assert(result == "test")
+        .select($"ret.f1").head().getString(0)
+    assert(result === "test")
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
index fbc8704..62b2e89 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
@@ -20,9 +20,11 @@ package org.apache.spark.sql
 import scala.beans.{BeanInfo, BeanProperty}
 
 import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.test.TestSQLContext._
 import org.apache.spark.sql.types._
 
+
 @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT])
 private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable {
   override def equals(other: Any): Boolean = other match {
@@ -66,14 +68,14 @@ class UserDefinedTypeSuite extends QueryTest {
 
 
   test("register user type: MyDenseVector for MyLabeledPoint") {
-    val labels: RDD[Double] = pointsRDD.select('label).map { case Row(v: Double) => v }
+    val labels: RDD[Double] = pointsRDD.select('label).rdd.map { case Row(v: Double) => v }
     val labelsArrays: Array[Double] = labels.collect()
     assert(labelsArrays.size === 2)
     assert(labelsArrays.contains(1.0))
     assert(labelsArrays.contains(0.0))
 
     val features: RDD[MyDenseVector] =
-      pointsRDD.select('features).map { case Row(v: MyDenseVector) => v }
+      pointsRDD.select('features).rdd.map { case Row(v: MyDenseVector) => v }
     val featuresArrays: Array[MyDenseVector] = features.collect()
     assert(featuresArrays.size === 2)
     assert(featuresArrays.contains(new MyDenseVector(Array(0.1, 1.0))))

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
index e61f3c3..6f051df 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql.columnar
 
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.TestData._
 import org.apache.spark.sql.catalyst.expressions.Row
 import org.apache.spark.sql.test.TestSQLContext._

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/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 67007b8..be5e63c 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
@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution
 import org.scalatest.FunSuite
 
 import org.apache.spark.sql.{SQLConf, execution}
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.TestData._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans._
@@ -28,6 +29,7 @@ import org.apache.spark.sql.test.TestSQLContext._
 import org.apache.spark.sql.test.TestSQLContext.planner._
 import org.apache.spark.sql.types._
 
+
 class PlannerSuite extends FunSuite {
   test("unions are collapsed") {
     val query = testData.unionAll(testData).unionAll(testData).logicalPlan
@@ -40,7 +42,7 @@ class PlannerSuite extends FunSuite {
   }
 
   test("count is partially aggregated") {
-    val query = testData.groupBy('value)(Count('key)).queryExecution.analyzed
+    val query = testData.groupBy('value).agg(count('key)).queryExecution.analyzed
     val planned = HashAggregation(query).head
     val aggregations = planned.collect { case n if n.nodeName contains "Aggregate" => n }
 
@@ -48,14 +50,14 @@ class PlannerSuite extends FunSuite {
   }
 
   test("count distinct is partially aggregated") {
-    val query = testData.groupBy('value)(CountDistinct('key :: Nil)).queryExecution.analyzed
+    val query = testData.groupBy('value).agg(countDistinct('key)).queryExecution.analyzed
     val planned = HashAggregation(query)
     assert(planned.nonEmpty)
   }
 
   test("mixed aggregates are partially aggregated") {
     val query =
-      testData.groupBy('value)(Count('value), CountDistinct('key :: Nil)).queryExecution.analyzed
+      testData.groupBy('value).agg(count('value), countDistinct('key)).queryExecution.analyzed
     val planned = HashAggregation(query)
     assert(planned.nonEmpty)
   }
@@ -128,9 +130,9 @@ class PlannerSuite extends FunSuite {
     testData.limit(3).registerTempTable("tiny")
     sql("CACHE TABLE tiny")
 
-    val a = testData.as('a)
-    val b = table("tiny").as('b)
-    val planned = a.join(b, Inner, Some("a.key".attr === "b.key".attr)).queryExecution.executedPlan
+    val a = testData.as("a")
+    val b = table("tiny").as("b")
+    val planned = a.join(b, $"a.key" === $"b.key").queryExecution.executedPlan
 
     val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join }
     val shuffledHashJoins = planned.collect { case join: ShuffledHashJoin => join }


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


[3/5] spark git commit: [SPARK-5097][SQL] DataFrame

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
new file mode 100644
index 0000000..d0bb364
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
@@ -0,0 +1,596 @@
+/*
+* 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
+
+import scala.language.implicitConversions
+import scala.reflect.ClassTag
+import scala.collection.JavaConversions._
+
+import java.util.{ArrayList, List => JList}
+
+import com.fasterxml.jackson.core.JsonFactory
+import net.razorvine.pickle.Pickler
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.rdd.RDD
+import org.apache.spark.api.java.JavaRDD
+import org.apache.spark.api.python.SerDeUtil
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.sql.catalyst.ScalaReflection
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.{Literal => LiteralExpr}
+import org.apache.spark.sql.catalyst.plans.{JoinType, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.execution.{LogicalRDD, EvaluatePython}
+import org.apache.spark.sql.json.JsonRDD
+import org.apache.spark.sql.types.{NumericType, StructType}
+import org.apache.spark.util.Utils
+
+
+/**
+ * A collection of rows that have the same columns.
+ *
+ * A [[DataFrame]] is equivalent to a relational table in Spark SQL, and can be created using
+ * various functions in [[SQLContext]].
+ * {{{
+ *   val people = sqlContext.parquetFile("...")
+ * }}}
+ *
+ * Once created, it can be manipulated using the various domain-specific-language (DSL) functions
+ * defined in: [[DataFrame]] (this class), [[Column]], and [[dsl]] for Scala DSL.
+ *
+ * To select a column from the data frame, use the apply method:
+ * {{{
+ *   val ageCol = people("age")  // in Scala
+ *   Column ageCol = people.apply("age")  // in Java
+ * }}}
+ *
+ * Note that the [[Column]] type can also be manipulated through its various functions.
+ * {{
+ *   // The following creates a new column that increases everybody's age by 10.
+ *   people("age") + 10  // in Scala
+ * }}
+ *
+ * A more concrete example:
+ * {{{
+ *   // To create DataFrame using SQLContext
+ *   val people = sqlContext.parquetFile("...")
+ *   val department = sqlContext.parquetFile("...")
+ *
+ *   people.filter("age" > 30)
+ *     .join(department, people("deptId") === department("id"))
+ *     .groupBy(department("name"), "gender")
+ *     .agg(avg(people("salary")), max(people("age")))
+ * }}}
+ */
+// TODO: Improve documentation.
+class DataFrame protected[sql](
+    val sqlContext: SQLContext,
+    private val baseLogicalPlan: LogicalPlan,
+    operatorsEnabled: Boolean)
+  extends DataFrameSpecificApi with RDDApi[Row] {
+
+  protected[sql] def this(sqlContext: Option[SQLContext], plan: Option[LogicalPlan]) =
+    this(sqlContext.orNull, plan.orNull, sqlContext.isDefined && plan.isDefined)
+
+  protected[sql] def this(sqlContext: SQLContext, plan: LogicalPlan) = this(sqlContext, plan, true)
+
+  @transient protected[sql] lazy val queryExecution = sqlContext.executePlan(baseLogicalPlan)
+
+  @transient protected[sql] val logicalPlan: LogicalPlan = baseLogicalPlan match {
+    // For various commands (like DDL) and queries with side effects, we force query optimization to
+    // happen right away to let these side effects take place eagerly.
+    case _: Command | _: InsertIntoTable | _: CreateTableAsSelect[_] |_: WriteToFile =>
+      LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext)
+    case _ =>
+      baseLogicalPlan
+  }
+
+  /**
+   * An implicit conversion function internal to this class for us to avoid doing
+   * "new DataFrame(...)" everywhere.
+   */
+  private[this] implicit def toDataFrame(logicalPlan: LogicalPlan): DataFrame = {
+    new DataFrame(sqlContext, logicalPlan, true)
+  }
+
+  /** Return the list of numeric columns, useful for doing aggregation. */
+  protected[sql] def numericColumns: Seq[Expression] = {
+    schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n =>
+      logicalPlan.resolve(n.name, sqlContext.analyzer.resolver).get
+    }
+  }
+
+  /** Resolve a column name into a Catalyst [[NamedExpression]]. */
+  protected[sql] def resolve(colName: String): NamedExpression = {
+    logicalPlan.resolve(colName, sqlContext.analyzer.resolver).getOrElse(
+      throw new RuntimeException(s"""Cannot resolve column name "$colName""""))
+  }
+
+  /** Left here for compatibility reasons. */
+  @deprecated("1.3.0", "use toDataFrame")
+  def toSchemaRDD: DataFrame = this
+
+  /**
+   * Return the object itself. Used to force an implicit conversion from RDD to DataFrame in Scala.
+   */
+  def toDF: DataFrame = this
+
+  /** Return the schema of this [[DataFrame]]. */
+  override def schema: StructType = queryExecution.analyzed.schema
+
+  /** Return all column names and their data types as an array. */
+  override def dtypes: Array[(String, String)] = schema.fields.map { field =>
+    (field.name, field.dataType.toString)
+  }
+
+  /** Return all column names as an array. */
+  override def columns: Array[String] = schema.fields.map(_.name)
+
+  /** Print the schema to the console in a nice tree format. */
+  override def printSchema(): Unit = println(schema.treeString)
+
+  /**
+   * Cartesian join with another [[DataFrame]].
+   *
+   * Note that cartesian joins are very expensive without an extra filter that can be pushed down.
+   *
+   * @param right Right side of the join operation.
+   */
+  override def join(right: DataFrame): DataFrame = {
+    Join(logicalPlan, right.logicalPlan, joinType = Inner, None)
+  }
+
+  /**
+   * Inner join with another [[DataFrame]], using the given join expression.
+   *
+   * {{{
+   *   // The following two are equivalent:
+   *   df1.join(df2, $"df1Key" === $"df2Key")
+   *   df1.join(df2).where($"df1Key" === $"df2Key")
+   * }}}
+   */
+  override def join(right: DataFrame, joinExprs: Column): DataFrame = {
+    Join(logicalPlan, right.logicalPlan, Inner, Some(joinExprs.expr))
+  }
+
+  /**
+   * Join with another [[DataFrame]], usin  g the given join expression. The following performs
+   * a full outer join between `df1` and `df2`.
+   *
+   * {{{
+   *   df1.join(df2, "outer", $"df1Key" === $"df2Key")
+   * }}}
+   *
+   * @param right Right side of the join.
+   * @param joinExprs Join expression.
+   * @param joinType One of: `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`.
+   */
+  override def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = {
+    Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr))
+  }
+
+  /**
+   * Return a new [[DataFrame]] sorted by the specified column, in ascending column.
+   * {{{
+   *   // The following 3 are equivalent
+   *   df.sort("sortcol")
+   *   df.sort($"sortcol")
+   *   df.sort($"sortcol".asc)
+   * }}}
+   */
+  override def sort(colName: String): DataFrame = {
+    Sort(Seq(SortOrder(apply(colName).expr, Ascending)), global = true, logicalPlan)
+  }
+
+  /**
+   * Return a new [[DataFrame]] sorted by the given expressions. For example:
+   * {{{
+   *   df.sort($"col1", $"col2".desc)
+   * }}}
+   */
+  @scala.annotation.varargs
+  override def sort(sortExpr: Column, sortExprs: Column*): DataFrame = {
+    val sortOrder: Seq[SortOrder] = (sortExpr +: sortExprs).map { col =>
+      col.expr match {
+        case expr: SortOrder =>
+          expr
+        case expr: Expression =>
+          SortOrder(expr, Ascending)
+      }
+    }
+    Sort(sortOrder, global = true, logicalPlan)
+  }
+
+  /**
+   * Return a new [[DataFrame]] sorted by the given expressions.
+   * This is an alias of the `sort` function.
+   */
+  @scala.annotation.varargs
+  override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame = {
+    sort(sortExpr, sortExprs :_*)
+  }
+
+  /**
+   * Selecting a single column and return it as a [[Column]].
+   */
+  override def apply(colName: String): Column = {
+    val expr = resolve(colName)
+    new Column(Some(sqlContext), Some(Project(Seq(expr), logicalPlan)), expr)
+  }
+
+  /**
+   * Selecting a set of expressions, wrapped in a Product.
+   * {{{
+   *   // The following two are equivalent:
+   *   df.apply(($"colA", $"colB" + 1))
+   *   df.select($"colA", $"colB" + 1)
+   * }}}
+   */
+  override def apply(projection: Product): DataFrame = {
+    require(projection.productArity >= 1)
+    select(projection.productIterator.map {
+      case c: Column => c
+      case o: Any => new Column(Some(sqlContext), None, LiteralExpr(o))
+    }.toSeq :_*)
+  }
+
+  /**
+   * Alias the current [[DataFrame]].
+   */
+  override def as(name: String): DataFrame = Subquery(name, logicalPlan)
+
+  /**
+   * Selecting a set of expressions.
+   * {{{
+   *   df.select($"colA", $"colB" + 1)
+   * }}}
+   */
+  @scala.annotation.varargs
+  override def select(cols: Column*): DataFrame = {
+    val exprs = cols.zipWithIndex.map {
+      case (Column(expr: NamedExpression), _) =>
+        expr
+      case (Column(expr: Expression), _) =>
+        Alias(expr, expr.toString)()
+    }
+    Project(exprs.toSeq, logicalPlan)
+  }
+
+  /**
+   * Selecting a set of columns. This is a variant of `select` that can only select
+   * existing columns using column names (i.e. cannot construct expressions).
+   *
+   * {{{
+   *   // The following two are equivalent:
+   *   df.select("colA", "colB")
+   *   df.select($"colA", $"colB")
+   * }}}
+   */
+  @scala.annotation.varargs
+  override def select(col: String, cols: String*): DataFrame = {
+    select((col +: cols).map(new Column(_)) :_*)
+  }
+
+  /**
+   * Filtering rows using the given condition.
+   * {{{
+   *   // The following are equivalent:
+   *   peopleDf.filter($"age" > 15)
+   *   peopleDf.where($"age" > 15)
+   *   peopleDf($"age" > 15)
+   * }}}
+   */
+  override def filter(condition: Column): DataFrame = {
+    Filter(condition.expr, logicalPlan)
+  }
+
+  /**
+   * Filtering rows using the given condition. This is an alias for `filter`.
+   * {{{
+   *   // The following are equivalent:
+   *   peopleDf.filter($"age" > 15)
+   *   peopleDf.where($"age" > 15)
+   *   peopleDf($"age" > 15)
+   * }}}
+   */
+  override def where(condition: Column): DataFrame = filter(condition)
+
+  /**
+   * Filtering rows using the given condition. This is a shorthand meant for Scala.
+   * {{{
+   *   // The following are equivalent:
+   *   peopleDf.filter($"age" > 15)
+   *   peopleDf.where($"age" > 15)
+   *   peopleDf($"age" > 15)
+   * }}}
+   */
+  override def apply(condition: Column): DataFrame = filter(condition)
+
+  /**
+   * Group the [[DataFrame]] using the specified columns, so we can run aggregation on them.
+   * See [[GroupedDataFrame]] for all the available aggregate functions.
+   *
+   * {{{
+   *   // Compute the average for all numeric columns grouped by department.
+   *   df.groupBy($"department").avg()
+   *
+   *   // Compute the max age and average salary, grouped by department and gender.
+   *   df.groupBy($"department", $"gender").agg(Map(
+   *     "salary" -> "avg",
+   *     "age" -> "max"
+   *   ))
+   * }}}
+   */
+  @scala.annotation.varargs
+  override def groupBy(cols: Column*): GroupedDataFrame = {
+    new GroupedDataFrame(this, cols.map(_.expr))
+  }
+
+  /**
+   * Group the [[DataFrame]] using the specified columns, so we can run aggregation on them.
+   * See [[GroupedDataFrame]] for all the available aggregate functions.
+   *
+   * This is a variant of groupBy that can only group by existing columns using column names
+   * (i.e. cannot construct expressions).
+   *
+   * {{{
+   *   // Compute the average for all numeric columns grouped by department.
+   *   df.groupBy("department").avg()
+   *
+   *   // Compute the max age and average salary, grouped by department and gender.
+   *   df.groupBy($"department", $"gender").agg(Map(
+   *     "salary" -> "avg",
+   *     "age" -> "max"
+   *   ))
+   * }}}
+   */
+  @scala.annotation.varargs
+  override def groupBy(col1: String, cols: String*): GroupedDataFrame = {
+    val colNames: Seq[String] = col1 +: cols
+    new GroupedDataFrame(this, colNames.map(colName => resolve(colName)))
+  }
+
+  /**
+   * Aggregate on the entire [[DataFrame]] without groups.
+   * {{
+   *   // df.agg(...) is a shorthand for df.groupBy().agg(...)
+   *   df.agg(Map("age" -> "max", "salary" -> "avg"))
+   *   df.groupBy().agg(Map("age" -> "max", "salary" -> "avg"))
+   * }}
+   */
+  override def agg(exprs: Map[String, String]): DataFrame = groupBy().agg(exprs)
+
+  /**
+   * Aggregate on the entire [[DataFrame]] without groups.
+   * {{
+   *   // df.agg(...) is a shorthand for df.groupBy().agg(...)
+   *   df.agg(max($"age"), avg($"salary"))
+   *   df.groupBy().agg(max($"age"), avg($"salary"))
+   * }}
+   */
+  @scala.annotation.varargs
+  override def agg(expr: Column, exprs: Column*): DataFrame = groupBy().agg(expr, exprs :_*)
+
+  /**
+   * Return a new [[DataFrame]] by taking the first `n` rows. The difference between this function
+   * and `head` is that `head` returns an array while `limit` returns a new [[DataFrame]].
+   */
+  override def limit(n: Int): DataFrame = Limit(LiteralExpr(n), logicalPlan)
+
+  /**
+   * Return a new [[DataFrame]] containing union of rows in this frame and another frame.
+   * This is equivalent to `UNION ALL` in SQL.
+   */
+  override def unionAll(other: DataFrame): DataFrame = Union(logicalPlan, other.logicalPlan)
+
+  /**
+   * Return a new [[DataFrame]] containing rows only in both this frame and another frame.
+   * This is equivalent to `INTERSECT` in SQL.
+   */
+  override def intersect(other: DataFrame): DataFrame = Intersect(logicalPlan, other.logicalPlan)
+
+  /**
+   * Return a new [[DataFrame]] containing rows in this frame but not in another frame.
+   * This is equivalent to `EXCEPT` in SQL.
+   */
+  override def except(other: DataFrame): DataFrame = Except(logicalPlan, other.logicalPlan)
+
+  /**
+   * Return a new [[DataFrame]] by sampling a fraction of rows.
+   *
+   * @param withReplacement Sample with replacement or not.
+   * @param fraction Fraction of rows to generate.
+   * @param seed Seed for sampling.
+   */
+  override def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame = {
+    Sample(fraction, withReplacement, seed, logicalPlan)
+  }
+
+  /**
+   * Return a new [[DataFrame]] by sampling a fraction of rows, using a random seed.
+   *
+   * @param withReplacement Sample with replacement or not.
+   * @param fraction Fraction of rows to generate.
+   */
+  override def sample(withReplacement: Boolean, fraction: Double): DataFrame = {
+    sample(withReplacement, fraction, Utils.random.nextLong)
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * Return a new [[DataFrame]] by adding a column.
+   */
+  override def addColumn(colName: String, col: Column): DataFrame = {
+    select(Column("*"), col.as(colName))
+  }
+
+  /**
+   * Return the first `n` rows.
+   */
+  override def head(n: Int): Array[Row] = limit(n).collect()
+
+  /**
+   * Return the first row.
+   */
+  override def head(): Row = head(1).head
+
+  /**
+   * Return the first row. Alias for head().
+   */
+  override def first(): Row = head()
+
+  override def map[R: ClassTag](f: Row => R): RDD[R] = {
+    rdd.map(f)
+  }
+
+  override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] = {
+    rdd.mapPartitions(f)
+  }
+
+  /**
+   * Return the first `n` rows in the [[DataFrame]].
+   */
+  override def take(n: Int): Array[Row] = head(n)
+
+  /**
+   * Return an array that contains all of [[Row]]s in this [[DataFrame]].
+   */
+  override def collect(): Array[Row] = rdd.collect()
+
+  /**
+   * Return a Java list that contains all of [[Row]]s in this [[DataFrame]].
+   */
+  override def collectAsList(): java.util.List[Row] = java.util.Arrays.asList(rdd.collect() :_*)
+
+  /**
+   * Return the number of rows in the [[DataFrame]].
+   */
+  override def count(): Long = groupBy().count().rdd.collect().head.getLong(0)
+
+  /**
+   * Return a new [[DataFrame]] that has exactly `numPartitions` partitions.
+   */
+  override def repartition(numPartitions: Int): DataFrame = {
+    sqlContext.applySchema(rdd.repartition(numPartitions), schema)
+  }
+
+  override def persist(): this.type = {
+    sqlContext.cacheQuery(this)
+    this
+  }
+
+  override def persist(newLevel: StorageLevel): this.type = {
+    sqlContext.cacheQuery(this, None, newLevel)
+    this
+  }
+
+  override def unpersist(blocking: Boolean): this.type = {
+    sqlContext.tryUncacheQuery(this, blocking)
+    this
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+  // I/O
+  /////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * Return the content of the [[DataFrame]] as a [[RDD]] of [[Row]]s.
+   */
+  override def rdd: RDD[Row] = {
+    val schema = this.schema
+    queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, schema))
+  }
+
+  /**
+   * Registers this RDD as a temporary table using the given name.  The lifetime of this temporary
+   * table is tied to the [[SQLContext]] that was used to create this DataFrame.
+   *
+   * @group schema
+   */
+  override def registerTempTable(tableName: String): Unit = {
+    sqlContext.registerRDDAsTable(this, tableName)
+  }
+
+  /**
+   * Saves the contents of this [[DataFrame]] as a parquet file, preserving the schema.
+   * Files that are written out using this method can be read back in as a [[DataFrame]]
+   * using the `parquetFile` function in [[SQLContext]].
+   */
+  override def saveAsParquetFile(path: String): Unit = {
+    sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd
+  }
+
+  /**
+   * :: Experimental ::
+   * Creates a table from the the contents of this DataFrame.  This will fail if the table already
+   * exists.
+   *
+   * Note that this currently only works with DataFrame that are created from a HiveContext as
+   * there is no notion of a persisted catalog in a standard SQL context.  Instead you can write
+   * an RDD out to a parquet file, and then register that file as a table.  This "table" can then
+   * be the target of an `insertInto`.
+   */
+  @Experimental
+  override def saveAsTable(tableName: String): Unit = {
+    sqlContext.executePlan(
+      CreateTableAsSelect(None, tableName, logicalPlan, allowExisting = false)).toRdd
+  }
+
+  /**
+   * :: Experimental ::
+   * Adds the rows from this RDD to the specified table, optionally overwriting the existing data.
+   */
+  @Experimental
+  override def insertInto(tableName: String, overwrite: Boolean): Unit = {
+    sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)),
+      Map.empty, logicalPlan, overwrite)).toRdd
+  }
+
+  /**
+   * Return the content of the [[DataFrame]] as a RDD of JSON strings.
+   */
+  override def toJSON: RDD[String] = {
+    val rowSchema = this.schema
+    this.mapPartitions { iter =>
+      val jsonFactory = new JsonFactory()
+      iter.map(JsonRDD.rowToJSON(rowSchema, jsonFactory))
+    }
+  }
+
+  ////////////////////////////////////////////////////////////////////////////
+  // for Python API
+  ////////////////////////////////////////////////////////////////////////////
+  /**
+   * A helpful function for Py4j, convert a list of Column to an array
+   */
+  protected[sql] def toColumnArray(cols: JList[Column]): Array[Column] = {
+    cols.toList.toArray
+  }
+
+  /**
+   * Converts a JavaRDD to a PythonRDD.
+   */
+  protected[sql] def javaToPython: JavaRDD[Array[Byte]] = {
+    val fieldTypes = schema.fields.map(_.dataType)
+    val jrdd = rdd.map(EvaluatePython.rowToArray(_, fieldTypes)).toJavaRDD()
+    SerDeUtil.javaToPython(jrdd)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala
new file mode 100644
index 0000000..1f1e9bd
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala
@@ -0,0 +1,139 @@
+/*
+ * 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
+
+import scala.language.implicitConversions
+import scala.collection.JavaConversions._
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.{Literal => LiteralExpr}
+import org.apache.spark.sql.catalyst.plans.logical.Aggregate
+
+
+/**
+ * A set of methods for aggregations on a [[DataFrame]], created by [[DataFrame.groupBy]].
+ */
+class GroupedDataFrame protected[sql](df: DataFrame, groupingExprs: Seq[Expression])
+  extends GroupedDataFrameApi {
+
+  private[this] implicit def toDataFrame(aggExprs: Seq[NamedExpression]): DataFrame = {
+    val namedGroupingExprs = groupingExprs.map {
+      case expr: NamedExpression => expr
+      case expr: Expression => Alias(expr, expr.toString)()
+    }
+    new DataFrame(df.sqlContext,
+      Aggregate(groupingExprs, namedGroupingExprs ++ aggExprs, df.logicalPlan))
+  }
+
+  private[this] def aggregateNumericColumns(f: Expression => Expression): Seq[NamedExpression] = {
+    df.numericColumns.map { c =>
+      val a = f(c)
+      Alias(a, a.toString)()
+    }
+  }
+
+  private[this] def strToExpr(expr: String): (Expression => Expression) = {
+    expr.toLowerCase match {
+      case "avg" | "average" | "mean" => Average
+      case "max" => Max
+      case "min" => Min
+      case "sum" => Sum
+      case "count" | "size" => Count
+    }
+  }
+
+  /**
+   * Compute aggregates by specifying a map from column name to aggregate methods.
+   * The available aggregate methods are `avg`, `max`, `min`, `sum`, `count`.
+   * {{{
+   *   // Selects the age of the oldest employee and the aggregate expense for each department
+   *   df.groupBy("department").agg(Map(
+   *     "age" -> "max"
+   *     "sum" -> "expense"
+   *   ))
+   * }}}
+   */
+  override def agg(exprs: Map[String, String]): DataFrame = {
+    exprs.map { case (colName, expr) =>
+      val a = strToExpr(expr)(df(colName).expr)
+      Alias(a, a.toString)()
+    }.toSeq
+  }
+
+  /**
+   * Compute aggregates by specifying a map from column name to aggregate methods.
+   * The available aggregate methods are `avg`, `max`, `min`, `sum`, `count`.
+   * {{{
+   *   // Selects the age of the oldest employee and the aggregate expense for each department
+   *   df.groupBy("department").agg(Map(
+   *     "age" -> "max"
+   *     "sum" -> "expense"
+   *   ))
+   * }}}
+   */
+  def agg(exprs: java.util.Map[String, String]): DataFrame = {
+    agg(exprs.toMap)
+  }
+
+  /**
+   * Compute aggregates by specifying a series of aggregate columns.
+   * The available aggregate methods are defined in [[org.apache.spark.sql.dsl]].
+   * {{{
+   *   // Selects the age of the oldest employee and the aggregate expense for each department
+   *   import org.apache.spark.sql.dsl._
+   *   df.groupBy("department").agg(max($"age"), sum($"expense"))
+   * }}}
+   */
+  @scala.annotation.varargs
+  override def agg(expr: Column, exprs: Column*): DataFrame = {
+    val aggExprs = (expr +: exprs).map(_.expr).map {
+      case expr: NamedExpression => expr
+      case expr: Expression => Alias(expr, expr.toString)()
+    }
+
+    new DataFrame(df.sqlContext, Aggregate(groupingExprs, aggExprs, df.logicalPlan))
+  }
+
+  /** Count the number of rows for each group. */
+  override def count(): DataFrame = Seq(Alias(Count(LiteralExpr(1)), "count")())
+
+  /**
+   * Compute the average value for each numeric columns for each group. This is an alias for `avg`.
+   */
+  override def mean(): DataFrame = aggregateNumericColumns(Average)
+
+  /**
+   * Compute the max value for each numeric columns for each group.
+   */
+  override def max(): DataFrame = aggregateNumericColumns(Max)
+
+  /**
+   * Compute the mean value for each numeric columns for each group.
+   */
+  override def avg(): DataFrame = aggregateNumericColumns(Average)
+
+  /**
+   * Compute the min value for each numeric column for each group.
+   */
+  override def min(): DataFrame = aggregateNumericColumns(Min)
+
+  /**
+   * Compute the sum for each numeric columns for each group.
+   */
+  override def sum(): DataFrame = aggregateNumericColumns(Sum)
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/Literal.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Literal.scala b/sql/core/src/main/scala/org/apache/spark/sql/Literal.scala
new file mode 100644
index 0000000..08cd4d0
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Literal.scala
@@ -0,0 +1,98 @@
+/*
+ * 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
+
+import org.apache.spark.sql.catalyst.expressions.{Literal => LiteralExpr}
+import org.apache.spark.sql.types._
+
+object Literal {
+
+  /** Return a new boolean literal. */
+  def apply(literal: Boolean): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new byte literal. */
+  def apply(literal: Byte): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new short literal. */
+  def apply(literal: Short): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new int literal. */
+  def apply(literal: Int): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new long literal. */
+  def apply(literal: Long): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new float literal. */
+  def apply(literal: Float): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new double literal. */
+  def apply(literal: Double): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new string literal. */
+  def apply(literal: String): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new decimal literal. */
+  def apply(literal: BigDecimal): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new decimal literal. */
+  def apply(literal: java.math.BigDecimal): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new timestamp literal. */
+  def apply(literal: java.sql.Timestamp): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new date literal. */
+  def apply(literal: java.sql.Date): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new binary (byte array) literal. */
+  def apply(literal: Array[Byte]): Column = new Column(LiteralExpr(literal))
+
+  /** Return a new null literal. */
+  def apply(literal: Null): Column = new Column(LiteralExpr(null))
+
+  /**
+   * Return a Column expression representing the literal value. Throws an exception if the
+   * data type is not supported by SparkSQL.
+   */
+  protected[sql] def anyToLiteral(literal: Any): Column = {
+    // If the literal is a symbol, convert it into a Column.
+    if (literal.isInstanceOf[Symbol]) {
+      return dsl.symbolToColumn(literal.asInstanceOf[Symbol])
+    }
+
+    val literalExpr = literal match {
+      case v: Int => LiteralExpr(v, IntegerType)
+      case v: Long => LiteralExpr(v, LongType)
+      case v: Double => LiteralExpr(v, DoubleType)
+      case v: Float => LiteralExpr(v, FloatType)
+      case v: Byte => LiteralExpr(v, ByteType)
+      case v: Short => LiteralExpr(v, ShortType)
+      case v: String => LiteralExpr(v, StringType)
+      case v: Boolean => LiteralExpr(v, BooleanType)
+      case v: BigDecimal => LiteralExpr(Decimal(v), DecimalType.Unlimited)
+      case v: java.math.BigDecimal => LiteralExpr(Decimal(v), DecimalType.Unlimited)
+      case v: Decimal => LiteralExpr(v, DecimalType.Unlimited)
+      case v: java.sql.Timestamp => LiteralExpr(v, TimestampType)
+      case v: java.sql.Date => LiteralExpr(v, DateType)
+      case v: Array[Byte] => LiteralExpr(v, BinaryType)
+      case null => LiteralExpr(null, NullType)
+      case _ =>
+        throw new RuntimeException("Unsupported literal type " + literal.getClass + " " + literal)
+    }
+    new Column(literalExpr)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 0a22968..5030e68 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -30,7 +30,6 @@ import org.apache.spark.api.java.{JavaSparkContext, JavaRDD}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.analysis._
-import org.apache.spark.sql.catalyst.dsl.ExpressionConversions
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
@@ -43,7 +42,7 @@ import org.apache.spark.util.Utils
 
 /**
  * :: AlphaComponent ::
- * The entry point for running relational queries using Spark.  Allows the creation of [[SchemaRDD]]
+ * The entry point for running relational queries using Spark.  Allows the creation of [[DataFrame]]
  * objects and the execution of SQL queries.
  *
  * @groupname userf Spark SQL Functions
@@ -53,7 +52,6 @@ import org.apache.spark.util.Utils
 class SQLContext(@transient val sparkContext: SparkContext)
   extends org.apache.spark.Logging
   with CacheManager
-  with ExpressionConversions
   with Serializable {
 
   self =>
@@ -111,8 +109,8 @@ class SQLContext(@transient val sparkContext: SparkContext)
   }
 
   protected[sql] def executeSql(sql: String): this.QueryExecution = executePlan(parseSql(sql))
-  protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution =
-    new this.QueryExecution { val logical = plan }
+
+  protected[sql] def executePlan(plan: LogicalPlan) = new this.QueryExecution(plan)
 
   sparkContext.getConf.getAll.foreach {
     case (key, value) if key.startsWith("spark.sql") => setConf(key, value)
@@ -124,24 +122,24 @@ class SQLContext(@transient val sparkContext: SparkContext)
    *
    * @group userf
    */
-  implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]): SchemaRDD = {
+  implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]): DataFrame = {
     SparkPlan.currentContext.set(self)
     val attributeSeq = ScalaReflection.attributesFor[A]
     val schema = StructType.fromAttributes(attributeSeq)
     val rowRDD = RDDConversions.productToRowRdd(rdd, schema)
-    new SchemaRDD(this, LogicalRDD(attributeSeq, rowRDD)(self))
+    new DataFrame(this, LogicalRDD(attributeSeq, rowRDD)(self))
   }
 
   /**
-   * Convert a [[BaseRelation]] created for external data sources into a [[SchemaRDD]].
+   * Convert a [[BaseRelation]] created for external data sources into a [[DataFrame]].
    */
-  def baseRelationToSchemaRDD(baseRelation: BaseRelation): SchemaRDD = {
-    new SchemaRDD(this, LogicalRelation(baseRelation))
+  def baseRelationToSchemaRDD(baseRelation: BaseRelation): DataFrame = {
+    new DataFrame(this, LogicalRelation(baseRelation))
   }
 
   /**
    * :: DeveloperApi ::
-   * Creates a [[SchemaRDD]] from an [[RDD]] containing [[Row]]s by applying a schema to this RDD.
+   * Creates a [[DataFrame]] from an [[RDD]] containing [[Row]]s by applying a schema to this RDD.
    * It is important to make sure that the structure of every [[Row]] of the provided RDD matches
    * the provided schema. Otherwise, there will be runtime exception.
    * Example:
@@ -170,11 +168,11 @@ class SQLContext(@transient val sparkContext: SparkContext)
    * @group userf
    */
   @DeveloperApi
-  def applySchema(rowRDD: RDD[Row], schema: StructType): SchemaRDD = {
+  def applySchema(rowRDD: RDD[Row], schema: StructType): DataFrame = {
     // TODO: use MutableProjection when rowRDD is another SchemaRDD and the applied
     // schema differs from the existing schema on any field data type.
     val logicalPlan = LogicalRDD(schema.toAttributes, rowRDD)(self)
-    new SchemaRDD(this, logicalPlan)
+    new DataFrame(this, logicalPlan)
   }
 
   /**
@@ -183,7 +181,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
    * WARNING: Since there is no guaranteed ordering for fields in a Java Bean,
    *          SELECT * queries will return the columns in an undefined order.
    */
-  def applySchema(rdd: RDD[_], beanClass: Class[_]): SchemaRDD = {
+  def applySchema(rdd: RDD[_], beanClass: Class[_]): DataFrame = {
     val attributeSeq = getSchema(beanClass)
     val className = beanClass.getName
     val rowRdd = rdd.mapPartitions { iter =>
@@ -201,7 +199,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
         ) : Row
       }
     }
-    new SchemaRDD(this, LogicalRDD(attributeSeq, rowRdd)(this))
+    new DataFrame(this, LogicalRDD(attributeSeq, rowRdd)(this))
   }
 
   /**
@@ -210,35 +208,35 @@ class SQLContext(@transient val sparkContext: SparkContext)
    * WARNING: Since there is no guaranteed ordering for fields in a Java Bean,
    *          SELECT * queries will return the columns in an undefined order.
    */
-  def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): SchemaRDD = {
+  def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): DataFrame = {
     applySchema(rdd.rdd, beanClass)
   }
 
   /**
-   * Loads a Parquet file, returning the result as a [[SchemaRDD]].
+   * Loads a Parquet file, returning the result as a [[DataFrame]].
    *
    * @group userf
    */
-  def parquetFile(path: String): SchemaRDD =
-    new SchemaRDD(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this))
+  def parquetFile(path: String): DataFrame =
+    new DataFrame(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this))
 
   /**
-   * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]].
+   * Loads a JSON file (one object per line), returning the result as a [[DataFrame]].
    * It goes through the entire dataset once to determine the schema.
    *
    * @group userf
    */
-  def jsonFile(path: String): SchemaRDD = jsonFile(path, 1.0)
+  def jsonFile(path: String): DataFrame = jsonFile(path, 1.0)
 
   /**
    * :: Experimental ::
    * Loads a JSON file (one object per line) and applies the given schema,
-   * returning the result as a [[SchemaRDD]].
+   * returning the result as a [[DataFrame]].
    *
    * @group userf
    */
   @Experimental
-  def jsonFile(path: String, schema: StructType): SchemaRDD = {
+  def jsonFile(path: String, schema: StructType): DataFrame = {
     val json = sparkContext.textFile(path)
     jsonRDD(json, schema)
   }
@@ -247,29 +245,29 @@ class SQLContext(@transient val sparkContext: SparkContext)
    * :: Experimental ::
    */
   @Experimental
-  def jsonFile(path: String, samplingRatio: Double): SchemaRDD = {
+  def jsonFile(path: String, samplingRatio: Double): DataFrame = {
     val json = sparkContext.textFile(path)
     jsonRDD(json, samplingRatio)
   }
 
   /**
    * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a
-   * [[SchemaRDD]].
+   * [[DataFrame]].
    * It goes through the entire dataset once to determine the schema.
    *
    * @group userf
    */
-  def jsonRDD(json: RDD[String]): SchemaRDD = jsonRDD(json, 1.0)
+  def jsonRDD(json: RDD[String]): DataFrame = jsonRDD(json, 1.0)
 
   /**
    * :: Experimental ::
    * Loads an RDD[String] storing JSON objects (one object per record) and applies the given schema,
-   * returning the result as a [[SchemaRDD]].
+   * returning the result as a [[DataFrame]].
    *
    * @group userf
    */
   @Experimental
-  def jsonRDD(json: RDD[String], schema: StructType): SchemaRDD = {
+  def jsonRDD(json: RDD[String], schema: StructType): DataFrame = {
     val columnNameOfCorruptJsonRecord = conf.columnNameOfCorruptRecord
     val appliedSchema =
       Option(schema).getOrElse(
@@ -283,7 +281,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
    * :: Experimental ::
    */
   @Experimental
-  def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD = {
+  def jsonRDD(json: RDD[String], samplingRatio: Double): DataFrame = {
     val columnNameOfCorruptJsonRecord = conf.columnNameOfCorruptRecord
     val appliedSchema =
       JsonRDD.nullTypeToStringType(
@@ -298,8 +296,8 @@ class SQLContext(@transient val sparkContext: SparkContext)
    *
    * @group userf
    */
-  def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = {
-    catalog.registerTable(Seq(tableName), rdd.queryExecution.logical)
+  def registerRDDAsTable(rdd: DataFrame, tableName: String): Unit = {
+    catalog.registerTable(Seq(tableName), rdd.logicalPlan)
   }
 
   /**
@@ -321,17 +319,17 @@ class SQLContext(@transient val sparkContext: SparkContext)
    *
    * @group userf
    */
-  def sql(sqlText: String): SchemaRDD = {
+  def sql(sqlText: String): DataFrame = {
     if (conf.dialect == "sql") {
-      new SchemaRDD(this, parseSql(sqlText))
+      new DataFrame(this, parseSql(sqlText))
     } else {
       sys.error(s"Unsupported SQL dialect: ${conf.dialect}")
     }
   }
 
   /** Returns the specified table as a SchemaRDD */
-  def table(tableName: String): SchemaRDD =
-    new SchemaRDD(this, catalog.lookupRelation(Seq(tableName)))
+  def table(tableName: String): DataFrame =
+    new DataFrame(this, catalog.lookupRelation(Seq(tableName)))
 
   /**
    * A collection of methods that are considered experimental, but can be used to hook into
@@ -454,15 +452,14 @@ class SQLContext(@transient val sparkContext: SparkContext)
    * access to the intermediate phases of query execution for developers.
    */
   @DeveloperApi
-  protected abstract class QueryExecution {
-    def logical: LogicalPlan
+  protected class QueryExecution(val logical: LogicalPlan) {
 
-    lazy val analyzed = ExtractPythonUdfs(analyzer(logical))
-    lazy val withCachedData = useCachedData(analyzed)
-    lazy val optimizedPlan = optimizer(withCachedData)
+    lazy val analyzed: LogicalPlan = ExtractPythonUdfs(analyzer(logical))
+    lazy val withCachedData: LogicalPlan = useCachedData(analyzed)
+    lazy val optimizedPlan: LogicalPlan = optimizer(withCachedData)
 
     // TODO: Don't just pick the first one...
-    lazy val sparkPlan = {
+    lazy val sparkPlan: SparkPlan = {
       SparkPlan.currentContext.set(self)
       planner(optimizedPlan).next()
     }
@@ -512,7 +509,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
    */
   protected[sql] def applySchemaToPythonRDD(
       rdd: RDD[Array[Any]],
-      schemaString: String): SchemaRDD = {
+      schemaString: String): DataFrame = {
     val schema = parseDataType(schemaString).asInstanceOf[StructType]
     applySchemaToPythonRDD(rdd, schema)
   }
@@ -522,7 +519,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
    */
   protected[sql] def applySchemaToPythonRDD(
       rdd: RDD[Array[Any]],
-      schema: StructType): SchemaRDD = {
+      schema: StructType): DataFrame = {
 
     def needsConversion(dataType: DataType): Boolean = dataType match {
       case ByteType => true
@@ -549,7 +546,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
       iter.map { m => new GenericRow(m): Row}
     }
 
-    new SchemaRDD(this, LogicalRDD(schema.toAttributes, rowRdd)(self))
+    new DataFrame(this, LogicalRDD(schema.toAttributes, rowRdd)(self))
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala
deleted file mode 100644
index d1e21df..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala
+++ /dev/null
@@ -1,511 +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
-
-import java.util.{List => JList}
-
-import scala.collection.JavaConversions._
-
-import com.fasterxml.jackson.core.JsonFactory
-
-import net.razorvine.pickle.Pickler
-
-import org.apache.spark.{Dependency, OneToOneDependency, Partition, Partitioner, TaskContext}
-import org.apache.spark.annotation.{AlphaComponent, Experimental}
-import org.apache.spark.api.java.JavaRDD
-import org.apache.spark.api.python.SerDeUtil
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.ScalaReflection
-import org.apache.spark.sql.catalyst.analysis._
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.execution.{LogicalRDD, EvaluatePython}
-import org.apache.spark.sql.json.JsonRDD
-import org.apache.spark.sql.types.{BooleanType, StructType}
-import org.apache.spark.storage.StorageLevel
-
-/**
- * :: AlphaComponent ::
- * An RDD of [[Row]] objects that has an associated schema. In addition to standard RDD functions,
- * SchemaRDDs can be used in relational queries, as shown in the examples below.
- *
- * Importing a SQLContext brings an implicit into scope that automatically converts a standard RDD
- * whose elements are scala case classes into a SchemaRDD.  This conversion can also be done
- * explicitly using the `createSchemaRDD` function on a [[SQLContext]].
- *
- * A `SchemaRDD` can also be created by loading data in from external sources.
- * Examples are loading data from Parquet files by using the `parquetFile` method on [[SQLContext]]
- * and loading JSON datasets by using `jsonFile` and `jsonRDD` methods on [[SQLContext]].
- *
- * == SQL Queries ==
- * A SchemaRDD can be registered as a table in the [[SQLContext]] that was used to create it.  Once
- * an RDD has been registered as a table, it can be used in the FROM clause of SQL statements.
- *
- * {{{
- *  // One method for defining the schema of an RDD is to make a case class with the desired column
- *  // names and types.
- *  case class Record(key: Int, value: String)
- *
- *  val sc: SparkContext // An existing spark context.
- *  val sqlContext = new SQLContext(sc)
- *
- *  // Importing the SQL context gives access to all the SQL functions and implicit conversions.
- *  import sqlContext._
- *
- *  val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i")))
- *  // Any RDD containing case classes can be registered as a table.  The schema of the table is
- *  // automatically inferred using scala reflection.
- *  rdd.registerTempTable("records")
- *
- *  val results: SchemaRDD = sql("SELECT * FROM records")
- * }}}
- *
- * == Language Integrated Queries ==
- *
- * {{{
- *
- *  case class Record(key: Int, value: String)
- *
- *  val sc: SparkContext // An existing spark context.
- *  val sqlContext = new SQLContext(sc)
- *
- *  // Importing the SQL context gives access to all the SQL functions and implicit conversions.
- *  import sqlContext._
- *
- *  val rdd = sc.parallelize((1 to 100).map(i => Record(i, "val_" + i)))
- *
- *  // Example of language integrated queries.
- *  rdd.where('key === 1).orderBy('value.asc).select('key).collect()
- * }}}
- *
- *  @groupname Query Language Integrated Queries
- *  @groupdesc Query Functions that create new queries from SchemaRDDs.  The
- *             result of all query functions is also a SchemaRDD, allowing multiple operations to be
- *             chained using a builder pattern.
- *  @groupprio Query -2
- *  @groupname schema SchemaRDD Functions
- *  @groupprio schema -1
- *  @groupname Ungrouped Base RDD Functions
- */
-@AlphaComponent
-class SchemaRDD(
-    @transient val sqlContext: SQLContext,
-    @transient val baseLogicalPlan: LogicalPlan)
-  extends RDD[Row](sqlContext.sparkContext, Nil) with SchemaRDDLike {
-
-  def baseSchemaRDD = this
-
-  // =========================================================================================
-  // RDD functions: Copy the internal row representation so we present immutable data to users.
-  // =========================================================================================
-
-  override def compute(split: Partition, context: TaskContext): Iterator[Row] =
-    firstParent[Row].compute(split, context).map(ScalaReflection.convertRowToScala(_, this.schema))
-
-  override def getPartitions: Array[Partition] = firstParent[Row].partitions
-
-  override protected def getDependencies: Seq[Dependency[_]] = {
-    schema // Force reification of the schema so it is available on executors.
-
-    List(new OneToOneDependency(queryExecution.toRdd))
-  }
-
-  /**
-   * Returns the schema of this SchemaRDD (represented by a [[StructType]]).
-   *
-   * @group schema
-   */
-  lazy val schema: StructType = queryExecution.analyzed.schema
-
-  /**
-   * Returns a new RDD with each row transformed to a JSON string.
-   *
-   * @group schema
-   */
-  def toJSON: RDD[String] = {
-    val rowSchema = this.schema
-    this.mapPartitions { iter =>
-      val jsonFactory = new JsonFactory()
-      iter.map(JsonRDD.rowToJSON(rowSchema, jsonFactory))
-    }
-  }
-
-
-  // =======================================================================
-  // Query DSL
-  // =======================================================================
-
-  /**
-   * Changes the output of this relation to the given expressions, similar to the `SELECT` clause
-   * in SQL.
-   *
-   * {{{
-   *   schemaRDD.select('a, 'b + 'c, 'd as 'aliasedName)
-   * }}}
-   *
-   * @param exprs a set of logical expression that will be evaluated for each input row.
-   *
-   * @group Query
-   */
-  def select(exprs: Expression*): SchemaRDD = {
-    val aliases = exprs.zipWithIndex.map {
-      case (ne: NamedExpression, _) => ne
-      case (e, i) => Alias(e, s"c$i")()
-    }
-    new SchemaRDD(sqlContext, Project(aliases, logicalPlan))
-  }
-
-  /**
-   * Filters the output, only returning those rows where `condition` evaluates to true.
-   *
-   * {{{
-   *   schemaRDD.where('a === 'b)
-   *   schemaRDD.where('a === 1)
-   *   schemaRDD.where('a + 'b > 10)
-   * }}}
-   *
-   * @group Query
-   */
-  def where(condition: Expression): SchemaRDD =
-    new SchemaRDD(sqlContext, Filter(condition, logicalPlan))
-
-  /**
-   * Performs a relational join on two SchemaRDDs
-   *
-   * @param otherPlan the [[SchemaRDD]] that should be joined with this one.
-   * @param joinType One of `Inner`, `LeftOuter`, `RightOuter`, or `FullOuter`. Defaults to `Inner.`
-   * @param on       An optional condition for the join operation.  This is equivalent to the `ON`
-   *                 clause in standard SQL.  In the case of `Inner` joins, specifying a
-   *                 `condition` is equivalent to adding `where` clauses after the `join`.
-   *
-   * @group Query
-   */
-  def join(
-      otherPlan: SchemaRDD,
-      joinType: JoinType = Inner,
-      on: Option[Expression] = None): SchemaRDD =
-    new SchemaRDD(sqlContext, Join(logicalPlan, otherPlan.logicalPlan, joinType, on))
-
-  /**
-   * Sorts the results by the given expressions.
-   * {{{
-   *   schemaRDD.orderBy('a)
-   *   schemaRDD.orderBy('a, 'b)
-   *   schemaRDD.orderBy('a.asc, 'b.desc)
-   * }}}
-   *
-   * @group Query
-   */
-  def orderBy(sortExprs: SortOrder*): SchemaRDD =
-    new SchemaRDD(sqlContext, Sort(sortExprs, true, logicalPlan))
-
-  /**
-   * Sorts the results by the given expressions within partition.
-   * {{{
-   *   schemaRDD.sortBy('a)
-   *   schemaRDD.sortBy('a, 'b)
-   *   schemaRDD.sortBy('a.asc, 'b.desc)
-   * }}}
-   *
-   * @group Query
-   */
-  def sortBy(sortExprs: SortOrder*): SchemaRDD =
-    new SchemaRDD(sqlContext, Sort(sortExprs, false, logicalPlan))
-
-  @deprecated("use limit with integer argument", "1.1.0")
-  def limit(limitExpr: Expression): SchemaRDD =
-    new SchemaRDD(sqlContext, Limit(limitExpr, logicalPlan))
-
-  /**
-   * Limits the results by the given integer.
-   * {{{
-   *   schemaRDD.limit(10)
-   * }}}
-   * @group Query
-   */
-  def limit(limitNum: Int): SchemaRDD =
-    new SchemaRDD(sqlContext, Limit(Literal(limitNum), logicalPlan))
-
-  /**
-   * Performs a grouping followed by an aggregation.
-   *
-   * {{{
-   *   schemaRDD.groupBy('year)(Sum('sales) as 'totalSales)
-   * }}}
-   *
-   * @group Query
-   */
-  def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*): SchemaRDD = {
-    val aliasedExprs = aggregateExprs.map {
-      case ne: NamedExpression => ne
-      case e => Alias(e, e.toString)()
-    }
-    new SchemaRDD(sqlContext, Aggregate(groupingExprs, aliasedExprs, logicalPlan))
-  }
-
-  /**
-   * Performs an aggregation over all Rows in this RDD.
-   * This is equivalent to a groupBy with no grouping expressions.
-   *
-   * {{{
-   *   schemaRDD.aggregate(Sum('sales) as 'totalSales)
-   * }}}
-   *
-   * @group Query
-   */
-  def aggregate(aggregateExprs: Expression*): SchemaRDD = {
-    groupBy()(aggregateExprs: _*)
-  }
-
-  /**
-   * Applies a qualifier to the attributes of this relation.  Can be used to disambiguate attributes
-   * with the same name, for example, when performing self-joins.
-   *
-   * {{{
-   *   val x = schemaRDD.where('a === 1).as('x)
-   *   val y = schemaRDD.where('a === 2).as('y)
-   *   x.join(y).where("x.a".attr === "y.a".attr),
-   * }}}
-   *
-   * @group Query
-   */
-  def as(alias: Symbol) =
-    new SchemaRDD(sqlContext, Subquery(alias.name, logicalPlan))
-
-  /**
-   * Combines the tuples of two RDDs with the same schema, keeping duplicates.
-   *
-   * @group Query
-   */
-  def unionAll(otherPlan: SchemaRDD) =
-    new SchemaRDD(sqlContext, Union(logicalPlan, otherPlan.logicalPlan))
-
-  /**
-   * Performs a relational except on two SchemaRDDs
-   *
-   * @param otherPlan the [[SchemaRDD]] that should be excepted from this one.
-   *
-   * @group Query
-   */
-  def except(otherPlan: SchemaRDD): SchemaRDD =
-    new SchemaRDD(sqlContext, Except(logicalPlan, otherPlan.logicalPlan))
-
-  /**
-   * Performs a relational intersect on two SchemaRDDs
-   *
-   * @param otherPlan the [[SchemaRDD]] that should be intersected with this one.
-   *
-   * @group Query
-   */
-  def intersect(otherPlan: SchemaRDD): SchemaRDD =
-    new SchemaRDD(sqlContext, Intersect(logicalPlan, otherPlan.logicalPlan))
-
-  /**
-   * Filters tuples using a function over the value of the specified column.
-   *
-   * {{{
-   *   schemaRDD.where('a)((a: Int) => ...)
-   * }}}
-   *
-   * @group Query
-   */
-  def where[T1](arg1: Symbol)(udf: (T1) => Boolean) =
-    new SchemaRDD(
-      sqlContext,
-      Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan))
-
-  /**
-   * :: Experimental ::
-   * Returns a sampled version of the underlying dataset.
-   *
-   * @group Query
-   */
-  @Experimental
-  override
-  def sample(
-      withReplacement: Boolean = true,
-      fraction: Double,
-      seed: Long) =
-    new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan))
-
-  /**
-   * :: Experimental ::
-   * Return the number of elements in the RDD. Unlike the base RDD implementation of count, this
-   * implementation leverages the query optimizer to compute the count on the SchemaRDD, which
-   * supports features such as filter pushdown.
-   * 
-   * @group Query
-   */
-  @Experimental
-  override def count(): Long = aggregate(Count(Literal(1))).collect().head.getLong(0)
-
-  /**
-   * :: Experimental ::
-   * Applies the given Generator, or table generating function, to this relation.
-   *
-   * @param generator A table generating function.  The API for such functions is likely to change
-   *                  in future releases
-   * @param join when set to true, each output row of the generator is joined with the input row
-   *             that produced it.
-   * @param outer when set to true, at least one row will be produced for each input row, similar to
-   *              an `OUTER JOIN` in SQL.  When no output rows are produced by the generator for a
-   *              given row, a single row will be output, with `NULL` values for each of the
-   *              generated columns.
-   * @param alias an optional alias that can be used as qualifier for the attributes that are
-   *              produced by this generate operation.
-   *
-   * @group Query
-   */
-  @Experimental
-  def generate(
-      generator: Generator,
-      join: Boolean = false,
-      outer: Boolean = false,
-      alias: Option[String] = None) =
-    new SchemaRDD(sqlContext, Generate(generator, join, outer, alias, logicalPlan))
-
-  /**
-   * Returns this RDD as a SchemaRDD.  Intended primarily to force the invocation of the implicit
-   * conversion from a standard RDD to a SchemaRDD.
-   *
-   * @group schema
-   */
-  def toSchemaRDD = this
-
-  /**
-   * Converts a JavaRDD to a PythonRDD. It is used by pyspark.
-   */
-  private[sql] def javaToPython: JavaRDD[Array[Byte]] = {
-    val fieldTypes = schema.fields.map(_.dataType)
-    val jrdd = this.map(EvaluatePython.rowToArray(_, fieldTypes)).toJavaRDD()
-    SerDeUtil.javaToPython(jrdd)
-  }
-
-  /**
-   * Serializes the Array[Row] returned by SchemaRDD's optimized collect(), using the same
-   * format as javaToPython. It is used by pyspark.
-   */
-  private[sql] def collectToPython: JList[Array[Byte]] = {
-    val fieldTypes = schema.fields.map(_.dataType)
-    val pickle = new Pickler
-    new java.util.ArrayList(collect().map { row =>
-      EvaluatePython.rowToArray(row, fieldTypes)
-    }.grouped(100).map(batched => pickle.dumps(batched.toArray)).toIterable)
-  }
-
-  /**
-   * Serializes the Array[Row] returned by SchemaRDD's takeSample(), using the same
-   * format as javaToPython and collectToPython. It is used by pyspark.
-   */
-  private[sql] def takeSampleToPython(
-      withReplacement: Boolean,
-      num: Int,
-      seed: Long): JList[Array[Byte]] = {
-    val fieldTypes = schema.fields.map(_.dataType)
-    val pickle = new Pickler
-    new java.util.ArrayList(this.takeSample(withReplacement, num, seed).map { row =>
-      EvaluatePython.rowToArray(row, fieldTypes)
-    }.grouped(100).map(batched => pickle.dumps(batched.toArray)).toIterable)
-  }
-
-  /**
-   * Creates SchemaRDD by applying own schema to derived RDD. Typically used to wrap return value
-   * of base RDD functions that do not change schema.
-   *
-   * @param rdd RDD derived from this one and has same schema
-   *
-   * @group schema
-   */
-  private def applySchema(rdd: RDD[Row]): SchemaRDD = {
-    new SchemaRDD(sqlContext,
-      LogicalRDD(queryExecution.analyzed.output.map(_.newInstance()), rdd)(sqlContext))
-  }
-
-  // =======================================================================
-  // Overridden RDD actions
-  // =======================================================================
-
-  override def collect(): Array[Row] = queryExecution.executedPlan.executeCollect()
-
-  def collectAsList(): java.util.List[Row] = java.util.Arrays.asList(collect() : _*)
-
-  override def take(num: Int): Array[Row] = limit(num).collect()
-
-  // =======================================================================
-  // Base RDD functions that do NOT change schema
-  // =======================================================================
-
-  // Transformations (return a new RDD)
-
-  override def coalesce(numPartitions: Int, shuffle: Boolean = false)
-                       (implicit ord: Ordering[Row] = null): SchemaRDD =
-    applySchema(super.coalesce(numPartitions, shuffle)(ord))
-
-  override def distinct(): SchemaRDD = applySchema(super.distinct())
-
-  override def distinct(numPartitions: Int)
-                       (implicit ord: Ordering[Row] = null): SchemaRDD =
-    applySchema(super.distinct(numPartitions)(ord))
-
-  def distinct(numPartitions: Int): SchemaRDD =
-    applySchema(super.distinct(numPartitions)(null))
-
-  override def filter(f: Row => Boolean): SchemaRDD =
-    applySchema(super.filter(f))
-
-  override def intersection(other: RDD[Row]): SchemaRDD =
-    applySchema(super.intersection(other))
-
-  override def intersection(other: RDD[Row], partitioner: Partitioner)
-                           (implicit ord: Ordering[Row] = null): SchemaRDD =
-    applySchema(super.intersection(other, partitioner)(ord))
-
-  override def intersection(other: RDD[Row], numPartitions: Int): SchemaRDD =
-    applySchema(super.intersection(other, numPartitions))
-
-  override def repartition(numPartitions: Int)
-                          (implicit ord: Ordering[Row] = null): SchemaRDD =
-    applySchema(super.repartition(numPartitions)(ord))
-
-  override def subtract(other: RDD[Row]): SchemaRDD =
-    applySchema(super.subtract(other))
-
-  override def subtract(other: RDD[Row], numPartitions: Int): SchemaRDD =
-    applySchema(super.subtract(other, numPartitions))
-
-  override def subtract(other: RDD[Row], p: Partitioner)
-                       (implicit ord: Ordering[Row] = null): SchemaRDD =
-    applySchema(super.subtract(other, p)(ord))
-
-  /** Overridden cache function will always use the in-memory columnar caching. */
-  override def cache(): this.type = {
-    sqlContext.cacheQuery(this)
-    this
-  }
-
-  override def persist(newLevel: StorageLevel): this.type = {
-    sqlContext.cacheQuery(this, None, newLevel)
-    this
-  }
-
-  override def unpersist(blocking: Boolean): this.type = {
-    sqlContext.tryUncacheQuery(this, blocking)
-    this
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala
deleted file mode 100644
index 3cf9209..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala
+++ /dev/null
@@ -1,139 +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
-
-import org.apache.spark.annotation.{DeveloperApi, Experimental}
-import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.execution.LogicalRDD
-
-/**
- * Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java)
- */
-private[sql] trait SchemaRDDLike {
-  @transient def sqlContext: SQLContext
-  @transient val baseLogicalPlan: LogicalPlan
-
-  private[sql] def baseSchemaRDD: SchemaRDD
-
-  /**
-   * :: DeveloperApi ::
-   * A lazily computed query execution workflow.  All other RDD operations are passed
-   * through to the RDD that is produced by this workflow. This workflow is produced lazily because
-   * invoking the whole query optimization pipeline can be expensive.
-   *
-   * The query execution is considered a Developer API as phases may be added or removed in future
-   * releases.  This execution is only exposed to provide an interface for inspecting the various
-   * phases for debugging purposes.  Applications should not depend on particular phases existing
-   * or producing any specific output, even for exactly the same query.
-   *
-   * Additionally, the RDD exposed by this execution is not designed for consumption by end users.
-   * In particular, it does not contain any schema information, and it reuses Row objects
-   * internally.  This object reuse improves performance, but can make programming against the RDD
-   * more difficult.  Instead end users should perform RDD operations on a SchemaRDD directly.
-   */
-  @transient
-  @DeveloperApi
-  lazy val queryExecution = sqlContext.executePlan(baseLogicalPlan)
-
-  @transient protected[spark] val logicalPlan: LogicalPlan = baseLogicalPlan match {
-    // For various commands (like DDL) and queries with side effects, we force query optimization to
-    // happen right away to let these side effects take place eagerly.
-    case _: Command | _: InsertIntoTable | _: CreateTableAsSelect[_] |_: WriteToFile =>
-      LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext)
-    case _ =>
-      baseLogicalPlan
-  }
-
-  override def toString =
-    s"""${super.toString}
-       |== Query Plan ==
-       |${queryExecution.simpleString}""".stripMargin.trim
-
-  /**
-   * Saves the contents of this `SchemaRDD` as a parquet file, preserving the schema.  Files that
-   * are written out using this method can be read back in as a SchemaRDD using the `parquetFile`
-   * function.
-   *
-   * @group schema
-   */
-  def saveAsParquetFile(path: String): Unit = {
-    sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd
-  }
-
-  /**
-   * Registers this RDD as a temporary table using the given name.  The lifetime of this temporary
-   * table is tied to the [[SQLContext]] that was used to create this SchemaRDD.
-   *
-   * @group schema
-   */
-  def registerTempTable(tableName: String): Unit = {
-    sqlContext.registerRDDAsTable(baseSchemaRDD, tableName)
-  }
-
-  @deprecated("Use registerTempTable instead of registerAsTable.", "1.1")
-  def registerAsTable(tableName: String): Unit = registerTempTable(tableName)
-
-  /**
-   * :: Experimental ::
-   * Adds the rows from this RDD to the specified table, optionally overwriting the existing data.
-   *
-   * @group schema
-   */
-  @Experimental
-  def insertInto(tableName: String, overwrite: Boolean): Unit =
-    sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)),
-      Map.empty, logicalPlan, overwrite)).toRdd
-
-  /**
-   * :: Experimental ::
-   * Appends the rows from this RDD to the specified table.
-   *
-   * @group schema
-   */
-  @Experimental
-  def insertInto(tableName: String): Unit = insertInto(tableName, overwrite = false)
-
-  /**
-   * :: Experimental ::
-   * Creates a table from the the contents of this SchemaRDD.  This will fail if the table already
-   * exists.
-   *
-   * Note that this currently only works with SchemaRDDs that are created from a HiveContext as
-   * there is no notion of a persisted catalog in a standard SQL context.  Instead you can write
-   * an RDD out to a parquet file, and then register that file as a table.  This "table" can then
-   * be the target of an `insertInto`.
-   *
-   * @group schema
-   */
-  @Experimental
-  def saveAsTable(tableName: String): Unit =
-    sqlContext.executePlan(CreateTableAsSelect(None, tableName, logicalPlan, false)).toRdd
-
-  /** Returns the schema as a string in the tree format.
-   *
-   * @group schema
-   */
-  def schemaString: String = baseSchemaRDD.schema.treeString
-
-  /** Prints out the schema.
-   *
-   * @group schema
-   */
-  def printSchema(): Unit = println(schemaString)
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/api.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api.scala b/sql/core/src/main/scala/org/apache/spark/sql/api.scala
new file mode 100644
index 0000000..073d41e
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api.scala
@@ -0,0 +1,289 @@
+/*
+* 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
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.api.java.JavaRDD
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.types.{DataType, StructType}
+import org.apache.spark.storage.StorageLevel
+
+
+/**
+ * An internal interface defining the RDD-like methods for [[DataFrame]].
+ * Please use [[DataFrame]] directly, and do NOT use this.
+ */
+trait RDDApi[T] {
+
+  def cache(): this.type = persist()
+
+  def persist(): this.type
+
+  def persist(newLevel: StorageLevel): this.type
+
+  def unpersist(): this.type = unpersist(blocking = false)
+
+  def unpersist(blocking: Boolean): this.type
+
+  def map[R: ClassTag](f: T => R): RDD[R]
+
+  def mapPartitions[R: ClassTag](f: Iterator[T] => Iterator[R]): RDD[R]
+
+  def take(n: Int): Array[T]
+
+  def collect(): Array[T]
+
+  def collectAsList(): java.util.List[T]
+
+  def count(): Long
+
+  def first(): T
+
+  def repartition(numPartitions: Int): DataFrame
+}
+
+
+/**
+ * An internal interface defining data frame related methods in [[DataFrame]].
+ * Please use [[DataFrame]] directly, and do NOT use this.
+ */
+trait DataFrameSpecificApi {
+
+  def schema: StructType
+
+  def printSchema(): Unit
+
+  def dtypes: Array[(String, String)]
+
+  def columns: Array[String]
+
+  def head(): Row
+
+  def head(n: Int): Array[Row]
+
+  /////////////////////////////////////////////////////////////////////////////
+  // Relational operators
+  /////////////////////////////////////////////////////////////////////////////
+  def apply(colName: String): Column
+
+  def apply(projection: Product): DataFrame
+
+  @scala.annotation.varargs
+  def select(cols: Column*): DataFrame
+
+  @scala.annotation.varargs
+  def select(col: String, cols: String*): DataFrame
+
+  def apply(condition: Column): DataFrame
+
+  def as(name: String): DataFrame
+
+  def filter(condition: Column): DataFrame
+
+  def where(condition: Column): DataFrame
+
+  @scala.annotation.varargs
+  def groupBy(cols: Column*): GroupedDataFrame
+
+  @scala.annotation.varargs
+  def groupBy(col1: String, cols: String*): GroupedDataFrame
+
+  def agg(exprs: Map[String, String]): DataFrame
+
+  @scala.annotation.varargs
+  def agg(expr: Column, exprs: Column*): DataFrame
+
+  def sort(colName: String): DataFrame
+
+  @scala.annotation.varargs
+  def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame
+
+  @scala.annotation.varargs
+  def sort(sortExpr: Column, sortExprs: Column*): DataFrame
+
+  def join(right: DataFrame): DataFrame
+
+  def join(right: DataFrame, joinExprs: Column): DataFrame
+
+  def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame
+
+  def limit(n: Int): DataFrame
+
+  def unionAll(other: DataFrame): DataFrame
+
+  def intersect(other: DataFrame): DataFrame
+
+  def except(other: DataFrame): DataFrame
+
+  def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame
+
+  def sample(withReplacement: Boolean, fraction: Double): DataFrame
+
+  /////////////////////////////////////////////////////////////////////////////
+  // Column mutation
+  /////////////////////////////////////////////////////////////////////////////
+  def addColumn(colName: String, col: Column): DataFrame
+
+  /////////////////////////////////////////////////////////////////////////////
+  // I/O and interaction with other frameworks
+  /////////////////////////////////////////////////////////////////////////////
+
+  def rdd: RDD[Row]
+
+  def toJavaRDD: JavaRDD[Row] = rdd.toJavaRDD()
+
+  def toJSON: RDD[String]
+
+  def registerTempTable(tableName: String): Unit
+
+  def saveAsParquetFile(path: String): Unit
+
+  @Experimental
+  def saveAsTable(tableName: String): Unit
+
+  @Experimental
+  def insertInto(tableName: String, overwrite: Boolean): Unit
+
+  @Experimental
+  def insertInto(tableName: String): Unit = insertInto(tableName, overwrite = false)
+
+  /////////////////////////////////////////////////////////////////////////////
+  // Stat functions
+  /////////////////////////////////////////////////////////////////////////////
+//  def describe(): Unit
+//
+//  def mean(): Unit
+//
+//  def max(): Unit
+//
+//  def min(): Unit
+}
+
+
+/**
+ * An internal interface defining expression APIs for [[DataFrame]].
+ * Please use [[DataFrame]] and [[Column]] directly, and do NOT use this.
+ */
+trait ExpressionApi {
+
+  def isComputable: Boolean
+
+  def unary_- : Column
+  def unary_! : Column
+  def unary_~ : Column
+
+  def + (other: Column): Column
+  def + (other: Any): Column
+  def - (other: Column): Column
+  def - (other: Any): Column
+  def * (other: Column): Column
+  def * (other: Any): Column
+  def / (other: Column): Column
+  def / (other: Any): Column
+  def % (other: Column): Column
+  def % (other: Any): Column
+  def & (other: Column): Column
+  def & (other: Any): Column
+  def | (other: Column): Column
+  def | (other: Any): Column
+  def ^ (other: Column): Column
+  def ^ (other: Any): Column
+
+  def && (other: Column): Column
+  def && (other: Boolean): Column
+  def || (other: Column): Column
+  def || (other: Boolean): Column
+
+  def < (other: Column): Column
+  def < (other: Any): Column
+  def <= (other: Column): Column
+  def <= (other: Any): Column
+  def > (other: Column): Column
+  def > (other: Any): Column
+  def >= (other: Column): Column
+  def >= (other: Any): Column
+  def === (other: Column): Column
+  def === (other: Any): Column
+  def equalTo(other: Column): Column
+  def equalTo(other: Any): Column
+  def <=> (other: Column): Column
+  def <=> (other: Any): Column
+  def !== (other: Column): Column
+  def !== (other: Any): Column
+
+  @scala.annotation.varargs
+  def in(list: Column*): Column
+
+  def like(other: Column): Column
+  def like(other: String): Column
+  def rlike(other: Column): Column
+  def rlike(other: String): Column
+
+  def contains(other: Column): Column
+  def contains(other: Any): Column
+  def startsWith(other: Column): Column
+  def startsWith(other: String): Column
+  def endsWith(other: Column): Column
+  def endsWith(other: String): Column
+
+  def substr(startPos: Column, len: Column): Column
+  def substr(startPos: Int, len: Int): Column
+
+  def isNull: Column
+  def isNotNull: Column
+
+  def getItem(ordinal: Column): Column
+  def getItem(ordinal: Int): Column
+  def getField(fieldName: String): Column
+
+  def cast(to: DataType): Column
+
+  def asc: Column
+  def desc: Column
+
+  def as(alias: String): Column
+}
+
+
+/**
+ * An internal interface defining aggregation APIs for [[DataFrame]].
+ * Please use [[DataFrame]] and [[GroupedDataFrame]] directly, and do NOT use this.
+ */
+trait GroupedDataFrameApi {
+
+  def agg(exprs: Map[String, String]): DataFrame
+
+  @scala.annotation.varargs
+  def agg(expr: Column, exprs: Column*): DataFrame
+
+  def avg(): DataFrame
+
+  def mean(): DataFrame
+
+  def min(): DataFrame
+
+  def max(): DataFrame
+
+  def sum(): DataFrame
+
+  def count(): DataFrame
+
+  // TODO: Add var, std
+}


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


[5/5] spark git commit: [SPARK-5097][SQL] DataFrame

Posted by rx...@apache.org.
[SPARK-5097][SQL] DataFrame

This pull request redesigns the existing Spark SQL dsl, which already provides data frame like functionalities.

TODOs:
With the exception of Python support, other tasks can be done in separate, follow-up PRs.
- [ ] Audit of the API
- [ ] Documentation
- [ ] More test cases to cover the new API
- [x] Python support
- [ ] Type alias SchemaRDD

Author: Reynold Xin <rx...@databricks.com>
Author: Davies Liu <da...@databricks.com>

Closes #4173 from rxin/df1 and squashes the following commits:

0a1a73b [Reynold Xin] Merge branch 'df1' of github.com:rxin/spark into df1
23b4427 [Reynold Xin] Mima.
828f70d [Reynold Xin] Merge pull request #7 from davies/df
257b9e6 [Davies Liu] add repartition
6bf2b73 [Davies Liu] fix collect with UDT and tests
e971078 [Reynold Xin] Missing quotes.
b9306b4 [Reynold Xin] Remove removeColumn/updateColumn for now.
a728bf2 [Reynold Xin] Example rename.
e8aa3d3 [Reynold Xin] groupby -> groupBy.
9662c9e [Davies Liu] improve DataFrame Python API
4ae51ea [Davies Liu] python API for dataframe
1e5e454 [Reynold Xin] Fixed a bug with symbol conversion.
2ca74db [Reynold Xin] Couple minor fixes.
ea98ea1 [Reynold Xin] Documentation & literal expressions.
2b22684 [Reynold Xin] Got rid of IntelliJ problems.
02bbfbc [Reynold Xin] Tightening imports.
ffbce66 [Reynold Xin] Fixed compilation error.
59b6d8b [Reynold Xin] Style violation.
b85edfb [Reynold Xin] ALS.
8c37f0a [Reynold Xin] Made MLlib and examples compile
6d53134 [Reynold Xin] Hive module.
d35efd5 [Reynold Xin] Fixed compilation error.
ce4a5d2 [Reynold Xin] Fixed test cases in SQL except ParquetIOSuite.
66d5ef1 [Reynold Xin] SQLContext minor patch.
c9bcdc0 [Reynold Xin] Checkpoint: SQL module compiles!


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

Branch: refs/heads/master
Commit: 119f45d61d7b48d376cca05e1b4f0c7fcf65bfa8
Parents: b1b35ca
Author: Reynold Xin <rx...@databricks.com>
Authored: Tue Jan 27 16:08:24 2015 -0800
Committer: Reynold Xin <rx...@databricks.com>
Committed: Tue Jan 27 16:08:24 2015 -0800

----------------------------------------------------------------------
 .../examples/ml/JavaCrossValidatorExample.java  |  10 +-
 .../examples/ml/JavaSimpleParamsExample.java    |  12 +-
 .../JavaSimpleTextClassificationPipeline.java   |  10 +-
 .../apache/spark/examples/sql/JavaSparkSQL.java |  36 +-
 .../src/main/python/mllib/dataset_example.py    |   2 +-
 examples/src/main/python/sql.py                 |  16 +-
 .../examples/ml/CrossValidatorExample.scala     |   3 +-
 .../apache/spark/examples/ml/MovieLensALS.scala |   2 +-
 .../spark/examples/ml/SimpleParamsExample.scala |   5 +-
 .../ml/SimpleTextClassificationPipeline.scala   |   3 +-
 .../spark/examples/mllib/DatasetExample.scala   |  28 +-
 .../apache/spark/examples/sql/RDDRelation.scala |   6 +-
 .../scala/org/apache/spark/ml/Estimator.scala   |   8 +-
 .../scala/org/apache/spark/ml/Evaluator.scala   |   4 +-
 .../scala/org/apache/spark/ml/Pipeline.scala    |   6 +-
 .../scala/org/apache/spark/ml/Transformer.scala |  17 +-
 .../ml/classification/LogisticRegression.scala  |  14 +-
 .../BinaryClassificationEvaluator.scala         |   7 +-
 .../spark/ml/feature/StandardScaler.scala       |  15 +-
 .../apache/spark/ml/recommendation/ALS.scala    |  37 +-
 .../apache/spark/ml/tuning/CrossValidator.scala |   8 +-
 .../org/apache/spark/ml/JavaPipelineSuite.java  |   6 +-
 .../JavaLogisticRegressionSuite.java            |   8 +-
 .../ml/tuning/JavaCrossValidatorSuite.java      |   4 +-
 .../org/apache/spark/ml/PipelineSuite.scala     |  14 +-
 .../LogisticRegressionSuite.scala               |  16 +-
 .../spark/ml/recommendation/ALSSuite.scala      |   4 +-
 .../spark/ml/tuning/CrossValidatorSuite.scala   |   4 +-
 project/MimaExcludes.scala                      |  15 +-
 python/pyspark/java_gateway.py                  |   7 +-
 python/pyspark/sql.py                           | 967 ++++++++++++++-----
 python/pyspark/tests.py                         | 155 +--
 .../analysis/MultiInstanceRelation.scala        |   2 +-
 .../catalyst/expressions/namedExpressions.scala |   3 +
 .../spark/sql/catalyst/plans/joinTypes.scala    |  15 +
 .../catalyst/plans/logical/TestRelation.scala   |   8 +-
 .../org/apache/spark/sql/CacheManager.scala     |   8 +-
 .../scala/org/apache/spark/sql/Column.scala     | 528 ++++++++++
 .../scala/org/apache/spark/sql/DataFrame.scala  | 596 ++++++++++++
 .../org/apache/spark/sql/GroupedDataFrame.scala | 139 +++
 .../scala/org/apache/spark/sql/Literal.scala    |  98 ++
 .../scala/org/apache/spark/sql/SQLContext.scala |  85 +-
 .../scala/org/apache/spark/sql/SchemaRDD.scala  | 511 ----------
 .../org/apache/spark/sql/SchemaRDDLike.scala    | 139 ---
 .../main/scala/org/apache/spark/sql/api.scala   | 289 ++++++
 .../org/apache/spark/sql/dsl/package.scala      | 495 ++++++++++
 .../apache/spark/sql/execution/commands.scala   |   8 +-
 .../spark/sql/execution/debug/package.scala     |   4 +-
 .../scala/org/apache/spark/sql/package.scala    |   2 +-
 .../apache/spark/sql/parquet/ParquetTest.scala  |   6 +-
 .../spark/sql/sources/DataSourceStrategy.scala  |  32 +-
 .../org/apache/spark/sql/sources/ddl.scala      |   5 +-
 .../apache/spark/sql/test/TestSQLContext.scala  |   6 +-
 .../apache/spark/sql/api/java/JavaAPISuite.java |   4 +-
 .../sql/api/java/JavaApplySchemaSuite.java      |  16 +-
 .../org/apache/spark/sql/CachedTableSuite.scala |   1 +
 .../org/apache/spark/sql/DslQuerySuite.scala    | 119 +--
 .../scala/org/apache/spark/sql/JoinSuite.scala  |  67 +-
 .../scala/org/apache/spark/sql/QueryTest.scala  |  12 +-
 .../org/apache/spark/sql/SQLQuerySuite.scala    |  18 +-
 .../scala/org/apache/spark/sql/TestData.scala   |  23 +-
 .../scala/org/apache/spark/sql/UDFSuite.scala   |  11 +-
 .../apache/spark/sql/UserDefinedTypeSuite.scala |   6 +-
 .../columnar/InMemoryColumnarQuerySuite.scala   |   1 +
 .../spark/sql/execution/PlannerSuite.scala      |  14 +-
 .../apache/spark/sql/execution/TgfSuite.scala   |  65 --
 .../org/apache/spark/sql/json/JsonSuite.scala   |  11 +-
 .../spark/sql/parquet/ParquetFilterSuite.scala  | 126 +--
 .../spark/sql/parquet/ParquetIOSuite.scala      |   7 +-
 .../spark/sql/sources/PrunedScanSuite.scala     |   2 +
 .../hive/thriftserver/SparkSQLCLIDriver.scala   |   2 +-
 .../spark/sql/hive/thriftserver/Shim12.scala    |   6 +-
 .../spark/sql/hive/thriftserver/Shim13.scala    |   6 +-
 .../org/apache/spark/sql/hive/HiveContext.scala |   9 +-
 .../apache/spark/sql/hive/HiveStrategies.scala  |  17 +-
 .../org/apache/spark/sql/hive/TestHive.scala    |   9 +-
 .../scala/org/apache/spark/sql/QueryTest.scala  |  10 +-
 .../spark/sql/hive/CachedTableSuite.scala       |   4 +-
 .../sql/hive/InsertIntoHiveTableSuite.scala     |   2 +-
 .../sql/hive/execution/HiveQuerySuite.scala     |   7 +-
 .../sql/hive/execution/HiveTableScanSuite.scala |  11 +-
 .../spark/sql/hive/execution/HiveUdfSuite.scala |   2 +-
 82 files changed, 3444 insertions(+), 1572 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java
index 247d2a5..0fbee6e 100644
--- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java
+++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java
@@ -33,7 +33,7 @@ import org.apache.spark.ml.param.ParamMap;
 import org.apache.spark.ml.tuning.CrossValidator;
 import org.apache.spark.ml.tuning.CrossValidatorModel;
 import org.apache.spark.ml.tuning.ParamGridBuilder;
-import org.apache.spark.sql.SchemaRDD;
+import org.apache.spark.sql.DataFrame;
 import org.apache.spark.sql.SQLContext;
 import org.apache.spark.sql.Row;
 
@@ -71,7 +71,7 @@ public class JavaCrossValidatorExample {
       new LabeledDocument(9L, "a e c l", 0.0),
       new LabeledDocument(10L, "spark compile", 1.0),
       new LabeledDocument(11L, "hadoop software", 0.0));
-    SchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class);
+    DataFrame training = jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class);
 
     // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr.
     Tokenizer tokenizer = new Tokenizer()
@@ -112,11 +112,11 @@ public class JavaCrossValidatorExample {
       new Document(5L, "l m n"),
       new Document(6L, "mapreduce spark"),
       new Document(7L, "apache hadoop"));
-    SchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class);
+    DataFrame test = jsql.applySchema(jsc.parallelize(localTest), Document.class);
 
     // Make predictions on test documents. cvModel uses the best model found (lrModel).
-    cvModel.transform(test).registerAsTable("prediction");
-    SchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction");
+    cvModel.transform(test).registerTempTable("prediction");
+    DataFrame predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction");
     for (Row r: predictions.collect()) {
       System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2)
           + ", prediction=" + r.get(3));

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java
index 5b92655..eaaa344 100644
--- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java
+++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java
@@ -28,7 +28,7 @@ import org.apache.spark.ml.param.ParamMap;
 import org.apache.spark.ml.classification.LogisticRegression;
 import org.apache.spark.mllib.linalg.Vectors;
 import org.apache.spark.mllib.regression.LabeledPoint;
-import org.apache.spark.sql.SchemaRDD;
+import org.apache.spark.sql.DataFrame;
 import org.apache.spark.sql.SQLContext;
 import org.apache.spark.sql.Row;
 
@@ -48,13 +48,13 @@ public class JavaSimpleParamsExample {
 
     // Prepare training data.
     // We use LabeledPoint, which is a JavaBean.  Spark SQL can convert RDDs of JavaBeans
-    // into SchemaRDDs, where it uses the bean metadata to infer the schema.
+    // into DataFrames, where it uses the bean metadata to infer the schema.
     List<LabeledPoint> localTraining = Lists.newArrayList(
       new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)),
       new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)),
       new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)),
       new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)));
-    SchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class);
+    DataFrame training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class);
 
     // Create a LogisticRegression instance.  This instance is an Estimator.
     LogisticRegression lr = new LogisticRegression();
@@ -94,14 +94,14 @@ public class JavaSimpleParamsExample {
         new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)),
         new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)),
         new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)));
-    SchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class);
+    DataFrame test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class);
 
     // Make predictions on test documents using the Transformer.transform() method.
     // LogisticRegression.transform will only use the 'features' column.
     // Note that model2.transform() outputs a 'probability' column instead of the usual 'score'
     // column since we renamed the lr.scoreCol parameter previously.
-    model2.transform(test).registerAsTable("results");
-    SchemaRDD results =
+    model2.transform(test).registerTempTable("results");
+    DataFrame results =
         jsql.sql("SELECT features, label, probability, prediction FROM results");
     for (Row r: results.collect()) {
       System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2)

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java
index 74db449..82d665a 100644
--- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java
+++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java
@@ -29,7 +29,7 @@ import org.apache.spark.ml.PipelineStage;
 import org.apache.spark.ml.classification.LogisticRegression;
 import org.apache.spark.ml.feature.HashingTF;
 import org.apache.spark.ml.feature.Tokenizer;
-import org.apache.spark.sql.SchemaRDD;
+import org.apache.spark.sql.DataFrame;
 import org.apache.spark.sql.SQLContext;
 import org.apache.spark.sql.Row;
 
@@ -54,7 +54,7 @@ public class JavaSimpleTextClassificationPipeline {
       new LabeledDocument(1L, "b d", 0.0),
       new LabeledDocument(2L, "spark f g h", 1.0),
       new LabeledDocument(3L, "hadoop mapreduce", 0.0));
-    SchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class);
+    DataFrame training = jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class);
 
     // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr.
     Tokenizer tokenizer = new Tokenizer()
@@ -79,11 +79,11 @@ public class JavaSimpleTextClassificationPipeline {
       new Document(5L, "l m n"),
       new Document(6L, "mapreduce spark"),
       new Document(7L, "apache hadoop"));
-    SchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class);
+    DataFrame test = jsql.applySchema(jsc.parallelize(localTest), Document.class);
 
     // Make predictions on test documents.
-    model.transform(test).registerAsTable("prediction");
-    SchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction");
+    model.transform(test).registerTempTable("prediction");
+    DataFrame predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction");
     for (Row r: predictions.collect()) {
       System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2)
           + ", prediction=" + r.get(3));

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java
index b708046..8defb76 100644
--- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java
+++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java
@@ -26,9 +26,9 @@ import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.api.java.function.Function;
 
-import org.apache.spark.sql.SQLContext;
-import org.apache.spark.sql.SchemaRDD;
+import org.apache.spark.sql.DataFrame;
 import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
 
 public class JavaSparkSQL {
   public static class Person implements Serializable {
@@ -74,13 +74,13 @@ public class JavaSparkSQL {
       });
 
     // Apply a schema to an RDD of Java Beans and register it as a table.
-    SchemaRDD schemaPeople = sqlCtx.applySchema(people, Person.class);
+    DataFrame schemaPeople = sqlCtx.applySchema(people, Person.class);
     schemaPeople.registerTempTable("people");
 
     // SQL can be run over RDDs that have been registered as tables.
-    SchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19");
+    DataFrame teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19");
 
-    // The results of SQL queries are SchemaRDDs and support all the normal RDD operations.
+    // The results of SQL queries are DataFrames and support all the normal RDD operations.
     // The columns of a row in the result can be accessed by ordinal.
     List<String> teenagerNames = teenagers.toJavaRDD().map(new Function<Row, String>() {
       @Override
@@ -93,17 +93,17 @@ public class JavaSparkSQL {
     }
 
     System.out.println("=== Data source: Parquet File ===");
-    // JavaSchemaRDDs can be saved as parquet files, maintaining the schema information.
+    // DataFrames can be saved as parquet files, maintaining the schema information.
     schemaPeople.saveAsParquetFile("people.parquet");
 
     // Read in the parquet file created above.
     // Parquet files are self-describing so the schema is preserved.
-    // The result of loading a parquet file is also a JavaSchemaRDD.
-    SchemaRDD parquetFile = sqlCtx.parquetFile("people.parquet");
+    // The result of loading a parquet file is also a DataFrame.
+    DataFrame parquetFile = sqlCtx.parquetFile("people.parquet");
 
     //Parquet files can also be registered as tables and then used in SQL statements.
     parquetFile.registerTempTable("parquetFile");
-    SchemaRDD teenagers2 =
+    DataFrame teenagers2 =
       sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19");
     teenagerNames = teenagers2.toJavaRDD().map(new Function<Row, String>() {
       @Override
@@ -119,8 +119,8 @@ public class JavaSparkSQL {
     // A JSON dataset is pointed by path.
     // The path can be either a single text file or a directory storing text files.
     String path = "examples/src/main/resources/people.json";
-    // Create a JavaSchemaRDD from the file(s) pointed by path
-    SchemaRDD peopleFromJsonFile = sqlCtx.jsonFile(path);
+    // Create a DataFrame from the file(s) pointed by path
+    DataFrame peopleFromJsonFile = sqlCtx.jsonFile(path);
 
     // Because the schema of a JSON dataset is automatically inferred, to write queries,
     // it is better to take a look at what is the schema.
@@ -130,13 +130,13 @@ public class JavaSparkSQL {
     //  |-- age: IntegerType
     //  |-- name: StringType
 
-    // Register this JavaSchemaRDD as a table.
+    // Register this DataFrame as a table.
     peopleFromJsonFile.registerTempTable("people");
 
     // SQL statements can be run by using the sql methods provided by sqlCtx.
-    SchemaRDD teenagers3 = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19");
+    DataFrame teenagers3 = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19");
 
-    // The results of SQL queries are JavaSchemaRDDs and support all the normal RDD operations.
+    // The results of SQL queries are DataFrame and support all the normal RDD operations.
     // The columns of a row in the result can be accessed by ordinal.
     teenagerNames = teenagers3.toJavaRDD().map(new Function<Row, String>() {
       @Override
@@ -146,14 +146,14 @@ public class JavaSparkSQL {
       System.out.println(name);
     }
 
-    // Alternatively, a JavaSchemaRDD can be created for a JSON dataset represented by
+    // Alternatively, a DataFrame can be created for a JSON dataset represented by
     // a RDD[String] storing one JSON object per string.
     List<String> jsonData = Arrays.asList(
           "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}");
     JavaRDD<String> anotherPeopleRDD = ctx.parallelize(jsonData);
-    SchemaRDD peopleFromJsonRDD = sqlCtx.jsonRDD(anotherPeopleRDD.rdd());
+    DataFrame peopleFromJsonRDD = sqlCtx.jsonRDD(anotherPeopleRDD.rdd());
 
-    // Take a look at the schema of this new JavaSchemaRDD.
+    // Take a look at the schema of this new DataFrame.
     peopleFromJsonRDD.printSchema();
     // The schema of anotherPeople is ...
     // root
@@ -164,7 +164,7 @@ public class JavaSparkSQL {
 
     peopleFromJsonRDD.registerTempTable("people2");
 
-    SchemaRDD peopleWithCity = sqlCtx.sql("SELECT name, address.city FROM people2");
+    DataFrame peopleWithCity = sqlCtx.sql("SELECT name, address.city FROM people2");
     List<String> nameAndCity = peopleWithCity.toJavaRDD().map(new Function<Row, String>() {
       @Override
       public String call(Row row) {

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/examples/src/main/python/mllib/dataset_example.py
----------------------------------------------------------------------
diff --git a/examples/src/main/python/mllib/dataset_example.py b/examples/src/main/python/mllib/dataset_example.py
index 540dae7..b5a70db 100644
--- a/examples/src/main/python/mllib/dataset_example.py
+++ b/examples/src/main/python/mllib/dataset_example.py
@@ -16,7 +16,7 @@
 #
 
 """
-An example of how to use SchemaRDD as a dataset for ML. Run with::
+An example of how to use DataFrame as a dataset for ML. Run with::
     bin/spark-submit examples/src/main/python/mllib/dataset_example.py
 """
 

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/examples/src/main/python/sql.py
----------------------------------------------------------------------
diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py
index d2c5ca4..7f5c68e 100644
--- a/examples/src/main/python/sql.py
+++ b/examples/src/main/python/sql.py
@@ -30,18 +30,18 @@ if __name__ == "__main__":
     some_rdd = sc.parallelize([Row(name="John", age=19),
                               Row(name="Smith", age=23),
                               Row(name="Sarah", age=18)])
-    # Infer schema from the first row, create a SchemaRDD and print the schema
-    some_schemardd = sqlContext.inferSchema(some_rdd)
-    some_schemardd.printSchema()
+    # Infer schema from the first row, create a DataFrame and print the schema
+    some_df = sqlContext.inferSchema(some_rdd)
+    some_df.printSchema()
 
     # Another RDD is created from a list of tuples
     another_rdd = sc.parallelize([("John", 19), ("Smith", 23), ("Sarah", 18)])
     # Schema with two fields - person_name and person_age
     schema = StructType([StructField("person_name", StringType(), False),
                         StructField("person_age", IntegerType(), False)])
-    # Create a SchemaRDD by applying the schema to the RDD and print the schema
-    another_schemardd = sqlContext.applySchema(another_rdd, schema)
-    another_schemardd.printSchema()
+    # Create a DataFrame by applying the schema to the RDD and print the schema
+    another_df = sqlContext.applySchema(another_rdd, schema)
+    another_df.printSchema()
     # root
     #  |-- age: integer (nullable = true)
     #  |-- name: string (nullable = true)
@@ -49,7 +49,7 @@ if __name__ == "__main__":
     # A JSON dataset is pointed to by path.
     # The path can be either a single text file or a directory storing text files.
     path = os.path.join(os.environ['SPARK_HOME'], "examples/src/main/resources/people.json")
-    # Create a SchemaRDD from the file(s) pointed to by path
+    # Create a DataFrame from the file(s) pointed to by path
     people = sqlContext.jsonFile(path)
     # root
     #  |-- person_name: string (nullable = false)
@@ -61,7 +61,7 @@ if __name__ == "__main__":
     #  |-- age: IntegerType
     #  |-- name: StringType
 
-    # Register this SchemaRDD as a table.
+    # Register this DataFrame as a table.
     people.registerAsTable("people")
 
     # SQL statements can be run by using the sql methods provided by sqlContext

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
index d8c7ef3..283bb80 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
@@ -18,7 +18,6 @@
 package org.apache.spark.examples.ml
 
 import org.apache.spark.{SparkConf, SparkContext}
-import org.apache.spark.SparkContext._
 import org.apache.spark.ml.Pipeline
 import org.apache.spark.ml.classification.LogisticRegression
 import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator
@@ -101,7 +100,7 @@ object CrossValidatorExample {
 
     // Make predictions on test documents. cvModel uses the best model found (lrModel).
     cvModel.transform(test)
-      .select('id, 'text, 'score, 'prediction)
+      .select("id", "text", "score", "prediction")
       .collect()
       .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) =>
       println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction)

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
index cf62772..b788582 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
@@ -143,7 +143,7 @@ object MovieLensALS {
 
     // Evaluate the model.
     // TODO: Create an evaluator to compute RMSE.
-    val mse = predictions.select('rating, 'prediction)
+    val mse = predictions.select("rating", "prediction").rdd
       .flatMap { case Row(rating: Float, prediction: Float) =>
         val err = rating.toDouble - prediction
         val err2 = err * err

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
index e8a2adf..95cc980 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
@@ -18,7 +18,6 @@
 package org.apache.spark.examples.ml
 
 import org.apache.spark.{SparkConf, SparkContext}
-import org.apache.spark.SparkContext._
 import org.apache.spark.ml.classification.LogisticRegression
 import org.apache.spark.ml.param.ParamMap
 import org.apache.spark.mllib.linalg.{Vector, Vectors}
@@ -42,7 +41,7 @@ object SimpleParamsExample {
 
     // Prepare training data.
     // We use LabeledPoint, which is a case class.  Spark SQL can convert RDDs of Java Beans
-    // into SchemaRDDs, where it uses the bean metadata to infer the schema.
+    // into DataFrames, where it uses the bean metadata to infer the schema.
     val training = sparkContext.parallelize(Seq(
       LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)),
       LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)),
@@ -92,7 +91,7 @@ object SimpleParamsExample {
     // Note that model2.transform() outputs a 'probability' column instead of the usual 'score'
     // column since we renamed the lr.scoreCol parameter previously.
     model2.transform(test)
-      .select('features, 'label, 'probability, 'prediction)
+      .select("features", "label", "probability", "prediction")
       .collect()
       .foreach { case Row(features: Vector, label: Double, prob: Double, prediction: Double) =>
         println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction)

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
index b9a6ef0..065db62 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
@@ -20,7 +20,6 @@ package org.apache.spark.examples.ml
 import scala.beans.BeanInfo
 
 import org.apache.spark.{SparkConf, SparkContext}
-import org.apache.spark.SparkContext._
 import org.apache.spark.ml.Pipeline
 import org.apache.spark.ml.classification.LogisticRegression
 import org.apache.spark.ml.feature.{HashingTF, Tokenizer}
@@ -80,7 +79,7 @@ object SimpleTextClassificationPipeline {
 
     // Make predictions on test documents.
     model.transform(test)
-      .select('id, 'text, 'score, 'prediction)
+      .select("id", "text", "score", "prediction")
       .collect()
       .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) =>
         println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction)

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
index f8d83f4..f229a58 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
@@ -28,10 +28,10 @@ import org.apache.spark.mllib.regression.LabeledPoint
 import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer
 import org.apache.spark.mllib.util.MLUtils
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{Row, SQLContext, SchemaRDD}
+import org.apache.spark.sql.{Row, SQLContext, DataFrame}
 
 /**
- * An example of how to use [[org.apache.spark.sql.SchemaRDD]] as a Dataset for ML. Run with
+ * An example of how to use [[org.apache.spark.sql.DataFrame]] as a Dataset for ML. Run with
  * {{{
  * ./bin/run-example org.apache.spark.examples.mllib.DatasetExample [options]
  * }}}
@@ -47,7 +47,7 @@ object DatasetExample {
     val defaultParams = Params()
 
     val parser = new OptionParser[Params]("DatasetExample") {
-      head("Dataset: an example app using SchemaRDD as a Dataset for ML.")
+      head("Dataset: an example app using DataFrame as a Dataset for ML.")
       opt[String]("input")
         .text(s"input path to dataset")
         .action((x, c) => c.copy(input = x))
@@ -80,20 +80,20 @@ object DatasetExample {
     }
     println(s"Loaded ${origData.count()} instances from file: ${params.input}")
 
-    // Convert input data to SchemaRDD explicitly.
-    val schemaRDD: SchemaRDD = origData
-    println(s"Inferred schema:\n${schemaRDD.schema.prettyJson}")
-    println(s"Converted to SchemaRDD with ${schemaRDD.count()} records")
+    // Convert input data to DataFrame explicitly.
+    val df: DataFrame = origData.toDF
+    println(s"Inferred schema:\n${df.schema.prettyJson}")
+    println(s"Converted to DataFrame with ${df.count()} records")
 
-    // Select columns, using implicit conversion to SchemaRDD.
-    val labelsSchemaRDD: SchemaRDD = origData.select('label)
-    val labels: RDD[Double] = labelsSchemaRDD.map { case Row(v: Double) => v }
+    // Select columns, using implicit conversion to DataFrames.
+    val labelsDf: DataFrame = origData.select("label")
+    val labels: RDD[Double] = labelsDf.map { case Row(v: Double) => v }
     val numLabels = labels.count()
     val meanLabel = labels.fold(0.0)(_ + _) / numLabels
     println(s"Selected label column with average value $meanLabel")
 
-    val featuresSchemaRDD: SchemaRDD = origData.select('features)
-    val features: RDD[Vector] = featuresSchemaRDD.map { case Row(v: Vector) => v }
+    val featuresDf: DataFrame = origData.select("features")
+    val features: RDD[Vector] = featuresDf.map { case Row(v: Vector) => v }
     val featureSummary = features.aggregate(new MultivariateOnlineSummarizer())(
       (summary, feat) => summary.add(feat),
       (sum1, sum2) => sum1.merge(sum2))
@@ -103,13 +103,13 @@ object DatasetExample {
     tmpDir.deleteOnExit()
     val outputDir = new File(tmpDir, "dataset").toString
     println(s"Saving to $outputDir as Parquet file.")
-    schemaRDD.saveAsParquetFile(outputDir)
+    df.saveAsParquetFile(outputDir)
 
     println(s"Loading Parquet file with UDT from $outputDir.")
     val newDataset = sqlContext.parquetFile(outputDir)
 
     println(s"Schema from Parquet: ${newDataset.schema.prettyJson}")
-    val newFeatures = newDataset.select('features).map { case Row(v: Vector) => v }
+    val newFeatures = newDataset.select("features").map { case Row(v: Vector) => v }
     val newFeaturesSummary = newFeatures.aggregate(new MultivariateOnlineSummarizer())(
       (summary, feat) => summary.add(feat),
       (sum1, sum2) => sum1.merge(sum2))

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
index 2e98b2d..a5d7f26 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
@@ -19,6 +19,8 @@ package org.apache.spark.examples.sql
 
 import org.apache.spark.{SparkConf, SparkContext}
 import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.dsl._
+import org.apache.spark.sql.dsl.literals._
 
 // One method for defining the schema of an RDD is to make a case class with the desired column
 // names and types.
@@ -54,7 +56,7 @@ object RDDRelation {
     rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect().foreach(println)
 
     // Queries can also be written using a LINQ-like Scala DSL.
-    rdd.where('key === 1).orderBy('value.asc).select('key).collect().foreach(println)
+    rdd.where($"key" === 1).orderBy($"value".asc).select($"key").collect().foreach(println)
 
     // Write out an RDD as a parquet file.
     rdd.saveAsParquetFile("pair.parquet")
@@ -63,7 +65,7 @@ object RDDRelation {
     val parquetFile = sqlContext.parquetFile("pair.parquet")
 
     // Queries can be run using the DSL on parequet files just like the original RDD.
-    parquetFile.where('key === 1).select('value as 'a).collect().foreach(println)
+    parquetFile.where($"key" === 1).select($"value".as("a")).collect().foreach(println)
 
     // These files can also be registered as tables.
     parquetFile.registerTempTable("parquetFile")

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala
index 77d230e..bc3defe 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala
@@ -21,7 +21,7 @@ import scala.annotation.varargs
 
 import org.apache.spark.annotation.AlphaComponent
 import org.apache.spark.ml.param.{ParamMap, ParamPair, Params}
-import org.apache.spark.sql.SchemaRDD
+import org.apache.spark.sql.DataFrame
 
 /**
  * :: AlphaComponent ::
@@ -38,7 +38,7 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params {
    * @return fitted model
    */
   @varargs
-  def fit(dataset: SchemaRDD, paramPairs: ParamPair[_]*): M = {
+  def fit(dataset: DataFrame, paramPairs: ParamPair[_]*): M = {
     val map = new ParamMap().put(paramPairs: _*)
     fit(dataset, map)
   }
@@ -50,7 +50,7 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params {
    * @param paramMap parameter map
    * @return fitted model
    */
-  def fit(dataset: SchemaRDD, paramMap: ParamMap): M
+  def fit(dataset: DataFrame, paramMap: ParamMap): M
 
   /**
    * Fits multiple models to the input data with multiple sets of parameters.
@@ -61,7 +61,7 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params {
    * @param paramMaps an array of parameter maps
    * @return fitted models, matching the input parameter maps
    */
-  def fit(dataset: SchemaRDD, paramMaps: Array[ParamMap]): Seq[M] = {
+  def fit(dataset: DataFrame, paramMaps: Array[ParamMap]): Seq[M] = {
     paramMaps.map(fit(dataset, _))
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/main/scala/org/apache/spark/ml/Evaluator.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/Evaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/Evaluator.scala
index db563dd..d2ca2e6 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/Evaluator.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/Evaluator.scala
@@ -19,7 +19,7 @@ package org.apache.spark.ml
 
 import org.apache.spark.annotation.AlphaComponent
 import org.apache.spark.ml.param.ParamMap
-import org.apache.spark.sql.SchemaRDD
+import org.apache.spark.sql.DataFrame
 
 /**
  * :: AlphaComponent ::
@@ -35,5 +35,5 @@ abstract class Evaluator extends Identifiable {
    * @param paramMap parameter map that specifies the input columns and output metrics
    * @return metric
    */
-  def evaluate(dataset: SchemaRDD, paramMap: ParamMap): Double
+  def evaluate(dataset: DataFrame, paramMap: ParamMap): Double
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
index ad6fed1..fe39cd1 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
@@ -22,7 +22,7 @@ import scala.collection.mutable.ListBuffer
 import org.apache.spark.Logging
 import org.apache.spark.annotation.AlphaComponent
 import org.apache.spark.ml.param.{Param, ParamMap}
-import org.apache.spark.sql.SchemaRDD
+import org.apache.spark.sql.DataFrame
 import org.apache.spark.sql.types.StructType
 
 /**
@@ -88,7 +88,7 @@ class Pipeline extends Estimator[PipelineModel] {
    * @param paramMap parameter map
    * @return fitted pipeline
    */
-  override def fit(dataset: SchemaRDD, paramMap: ParamMap): PipelineModel = {
+  override def fit(dataset: DataFrame, paramMap: ParamMap): PipelineModel = {
     transformSchema(dataset.schema, paramMap, logging = true)
     val map = this.paramMap ++ paramMap
     val theStages = map(stages)
@@ -162,7 +162,7 @@ class PipelineModel private[ml] (
     }
   }
 
-  override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = {
+  override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = {
     // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap
     val map = (fittingParamMap ++ this.paramMap) ++ paramMap
     transformSchema(dataset.schema, map, logging = true)

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala
index af56f9c..b233bff 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala
@@ -22,9 +22,9 @@ import scala.annotation.varargs
 import org.apache.spark.Logging
 import org.apache.spark.annotation.AlphaComponent
 import org.apache.spark.ml.param._
-import org.apache.spark.sql.SchemaRDD
-import org.apache.spark.sql.catalyst.analysis.Star
-import org.apache.spark.sql.catalyst.expressions.ScalaUdf
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql._
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.types._
 
 /**
@@ -41,7 +41,7 @@ abstract class Transformer extends PipelineStage with Params {
    * @return transformed dataset
    */
   @varargs
-  def transform(dataset: SchemaRDD, paramPairs: ParamPair[_]*): SchemaRDD = {
+  def transform(dataset: DataFrame, paramPairs: ParamPair[_]*): DataFrame = {
     val map = new ParamMap()
     paramPairs.foreach(map.put(_))
     transform(dataset, map)
@@ -53,7 +53,7 @@ abstract class Transformer extends PipelineStage with Params {
    * @param paramMap additional parameters, overwrite embedded params
    * @return transformed dataset
    */
-  def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD
+  def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame
 }
 
 /**
@@ -95,11 +95,10 @@ private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, O
     StructType(outputFields)
   }
 
-  override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = {
+  override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = {
     transformSchema(dataset.schema, paramMap, logging = true)
-    import dataset.sqlContext._
     val map = this.paramMap ++ paramMap
-    val udf = ScalaUdf(this.createTransformFunc(map), outputDataType, Seq(map(inputCol).attr))
-    dataset.select(Star(None), udf as map(outputCol))
+    dataset.select($"*", callUDF(
+      this.createTransformFunc(map), outputDataType, Column(map(inputCol))).as(map(outputCol)))
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
index 8c57081..eeb6301 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
@@ -24,7 +24,7 @@ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS
 import org.apache.spark.mllib.linalg.{BLAS, Vector, VectorUDT}
 import org.apache.spark.mllib.regression.LabeledPoint
 import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.analysis.Star
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.catalyst.dsl._
 import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
 import org.apache.spark.storage.StorageLevel
@@ -87,11 +87,10 @@ class LogisticRegression extends Estimator[LogisticRegressionModel] with Logisti
   def setScoreCol(value: String): this.type = set(scoreCol, value)
   def setPredictionCol(value: String): this.type = set(predictionCol, value)
 
-  override def fit(dataset: SchemaRDD, paramMap: ParamMap): LogisticRegressionModel = {
+  override def fit(dataset: DataFrame, paramMap: ParamMap): LogisticRegressionModel = {
     transformSchema(dataset.schema, paramMap, logging = true)
-    import dataset.sqlContext._
     val map = this.paramMap ++ paramMap
-    val instances = dataset.select(map(labelCol).attr, map(featuresCol).attr)
+    val instances = dataset.select(map(labelCol), map(featuresCol))
       .map { case Row(label: Double, features: Vector) =>
         LabeledPoint(label, features)
       }.persist(StorageLevel.MEMORY_AND_DISK)
@@ -131,9 +130,8 @@ class LogisticRegressionModel private[ml] (
     validateAndTransformSchema(schema, paramMap, fitting = false)
   }
 
-  override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = {
+  override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = {
     transformSchema(dataset.schema, paramMap, logging = true)
-    import dataset.sqlContext._
     val map = this.paramMap ++ paramMap
     val score: Vector => Double = (v) => {
       val margin = BLAS.dot(v, weights)
@@ -143,7 +141,7 @@ class LogisticRegressionModel private[ml] (
     val predict: Double => Double = (score) => {
       if (score > t) 1.0 else 0.0
     }
-    dataset.select(Star(None), score.call(map(featuresCol).attr) as map(scoreCol))
-      .select(Star(None), predict.call(map(scoreCol).attr) as map(predictionCol))
+    dataset.select($"*", callUDF(score, Column(map(featuresCol))).as(map(scoreCol)))
+      .select($"*", callUDF(predict, Column(map(scoreCol))).as(map(predictionCol)))
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala
index 12473cb..1979ab9 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala
@@ -21,7 +21,7 @@ import org.apache.spark.annotation.AlphaComponent
 import org.apache.spark.ml._
 import org.apache.spark.ml.param._
 import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics
-import org.apache.spark.sql.{Row, SchemaRDD}
+import org.apache.spark.sql.{DataFrame, Row}
 import org.apache.spark.sql.types.DoubleType
 
 /**
@@ -41,7 +41,7 @@ class BinaryClassificationEvaluator extends Evaluator with Params
   def setScoreCol(value: String): this.type = set(scoreCol, value)
   def setLabelCol(value: String): this.type = set(labelCol, value)
 
-  override def evaluate(dataset: SchemaRDD, paramMap: ParamMap): Double = {
+  override def evaluate(dataset: DataFrame, paramMap: ParamMap): Double = {
     val map = this.paramMap ++ paramMap
 
     val schema = dataset.schema
@@ -52,8 +52,7 @@ class BinaryClassificationEvaluator extends Evaluator with Params
     require(labelType == DoubleType,
       s"Label column ${map(labelCol)} must be double type but found $labelType")
 
-    import dataset.sqlContext._
-    val scoreAndLabels = dataset.select(map(scoreCol).attr, map(labelCol).attr)
+    val scoreAndLabels = dataset.select(map(scoreCol), map(labelCol))
       .map { case Row(score: Double, label: Double) =>
         (score, label)
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala
index 72825f6..e7bdb07 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala
@@ -23,7 +23,7 @@ import org.apache.spark.ml.param._
 import org.apache.spark.mllib.feature
 import org.apache.spark.mllib.linalg.{Vector, VectorUDT}
 import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.analysis.Star
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.catalyst.dsl._
 import org.apache.spark.sql.types.{StructField, StructType}
 
@@ -43,14 +43,10 @@ class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerP
   def setInputCol(value: String): this.type = set(inputCol, value)
   def setOutputCol(value: String): this.type = set(outputCol, value)
 
-  override def fit(dataset: SchemaRDD, paramMap: ParamMap): StandardScalerModel = {
+  override def fit(dataset: DataFrame, paramMap: ParamMap): StandardScalerModel = {
     transformSchema(dataset.schema, paramMap, logging = true)
-    import dataset.sqlContext._
     val map = this.paramMap ++ paramMap
-    val input = dataset.select(map(inputCol).attr)
-      .map { case Row(v: Vector) =>
-        v
-      }
+    val input = dataset.select(map(inputCol)).map { case Row(v: Vector) => v }
     val scaler = new feature.StandardScaler().fit(input)
     val model = new StandardScalerModel(this, map, scaler)
     Params.inheritValues(map, this, model)
@@ -83,14 +79,13 @@ class StandardScalerModel private[ml] (
   def setInputCol(value: String): this.type = set(inputCol, value)
   def setOutputCol(value: String): this.type = set(outputCol, value)
 
-  override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = {
+  override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = {
     transformSchema(dataset.schema, paramMap, logging = true)
-    import dataset.sqlContext._
     val map = this.paramMap ++ paramMap
     val scale: (Vector) => Vector = (v) => {
       scaler.transform(v)
     }
-    dataset.select(Star(None), scale.call(map(inputCol).attr) as map(outputCol))
+    dataset.select($"*", callUDF(scale, Column(map(inputCol))).as(map(outputCol)))
   }
 
   private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = {

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala
index 2d89e76..f6437c7 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala
@@ -29,10 +29,8 @@ import org.apache.spark.{HashPartitioner, Logging, Partitioner}
 import org.apache.spark.ml.{Estimator, Model}
 import org.apache.spark.ml.param._
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SchemaRDD
-import org.apache.spark.sql.catalyst.dsl._
-import org.apache.spark.sql.catalyst.expressions.Cast
-import org.apache.spark.sql.catalyst.plans.LeftOuter
+import org.apache.spark.sql.{Column, DataFrame}
+import org.apache.spark.sql.dsl._
 import org.apache.spark.sql.types.{DoubleType, FloatType, IntegerType, StructField, StructType}
 import org.apache.spark.util.Utils
 import org.apache.spark.util.collection.{OpenHashMap, OpenHashSet, SortDataFormat, Sorter}
@@ -112,7 +110,7 @@ class ALSModel private[ml] (
 
   def setPredictionCol(value: String): this.type = set(predictionCol, value)
 
-  override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = {
+  override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = {
     import dataset.sqlContext._
     import org.apache.spark.ml.recommendation.ALSModel.Factor
     val map = this.paramMap ++ paramMap
@@ -120,13 +118,13 @@ class ALSModel private[ml] (
     val instanceTable = s"instance_$uid"
     val userTable = s"user_$uid"
     val itemTable = s"item_$uid"
-    val instances = dataset.as(Symbol(instanceTable))
+    val instances = dataset.as(instanceTable)
     val users = userFactors.map { case (id, features) =>
       Factor(id, features)
-    }.as(Symbol(userTable))
+    }.as(userTable)
     val items = itemFactors.map { case (id, features) =>
       Factor(id, features)
-    }.as(Symbol(itemTable))
+    }.as(itemTable)
     val predict: (Seq[Float], Seq[Float]) => Float = (userFeatures, itemFeatures) => {
       if (userFeatures != null && itemFeatures != null) {
         blas.sdot(k, userFeatures.toArray, 1, itemFeatures.toArray, 1)
@@ -135,12 +133,12 @@ class ALSModel private[ml] (
       }
     }
     val inputColumns = dataset.schema.fieldNames
-    val prediction =
-      predict.call(s"$userTable.features".attr, s"$itemTable.features".attr) as map(predictionCol)
-    val outputColumns = inputColumns.map(f => s"$instanceTable.$f".attr as f) :+ prediction
+    val prediction = callUDF(predict, $"$userTable.features", $"$itemTable.features")
+        .as(map(predictionCol))
+    val outputColumns = inputColumns.map(f => $"$instanceTable.$f".as(f)) :+ prediction
     instances
-      .join(users, LeftOuter, Some(map(userCol).attr === s"$userTable.id".attr))
-      .join(items, LeftOuter, Some(map(itemCol).attr === s"$itemTable.id".attr))
+      .join(users, Column(map(userCol)) === $"$userTable.id", "left")
+      .join(items, Column(map(itemCol)) === $"$itemTable.id", "left")
       .select(outputColumns: _*)
   }
 
@@ -209,14 +207,13 @@ class ALS extends Estimator[ALSModel] with ALSParams {
   setMaxIter(20)
   setRegParam(1.0)
 
-  override def fit(dataset: SchemaRDD, paramMap: ParamMap): ALSModel = {
-    import dataset.sqlContext._
+  override def fit(dataset: DataFrame, paramMap: ParamMap): ALSModel = {
     val map = this.paramMap ++ paramMap
-    val ratings =
-      dataset.select(map(userCol).attr, map(itemCol).attr, Cast(map(ratingCol).attr, FloatType))
-        .map { row =>
-          new Rating(row.getInt(0), row.getInt(1), row.getFloat(2))
-        }
+    val ratings = dataset
+      .select(Column(map(userCol)), Column(map(itemCol)), Column(map(ratingCol)).cast(FloatType))
+      .map { row =>
+        new Rating(row.getInt(0), row.getInt(1), row.getFloat(2))
+      }
     val (userFactors, itemFactors) = ALS.train(ratings, rank = map(rank),
       numUserBlocks = map(numUserBlocks), numItemBlocks = map(numItemBlocks),
       maxIter = map(maxIter), regParam = map(regParam), implicitPrefs = map(implicitPrefs),

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala
index 08fe991..5d51c51 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala
@@ -24,7 +24,7 @@ import org.apache.spark.annotation.AlphaComponent
 import org.apache.spark.ml._
 import org.apache.spark.ml.param.{IntParam, Param, ParamMap, Params}
 import org.apache.spark.mllib.util.MLUtils
-import org.apache.spark.sql.SchemaRDD
+import org.apache.spark.sql.DataFrame
 import org.apache.spark.sql.types.StructType
 
 /**
@@ -64,7 +64,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP
   def setEvaluator(value: Evaluator): this.type = set(evaluator, value)
   def setNumFolds(value: Int): this.type = set(numFolds, value)
 
-  override def fit(dataset: SchemaRDD, paramMap: ParamMap): CrossValidatorModel = {
+  override def fit(dataset: DataFrame, paramMap: ParamMap): CrossValidatorModel = {
     val map = this.paramMap ++ paramMap
     val schema = dataset.schema
     transformSchema(dataset.schema, paramMap, logging = true)
@@ -74,7 +74,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP
     val epm = map(estimatorParamMaps)
     val numModels = epm.size
     val metrics = new Array[Double](epm.size)
-    val splits = MLUtils.kFold(dataset, map(numFolds), 0)
+    val splits = MLUtils.kFold(dataset.rdd, map(numFolds), 0)
     splits.zipWithIndex.foreach { case ((training, validation), splitIndex) =>
       val trainingDataset = sqlCtx.applySchema(training, schema).cache()
       val validationDataset = sqlCtx.applySchema(validation, schema).cache()
@@ -117,7 +117,7 @@ class CrossValidatorModel private[ml] (
     val bestModel: Model[_])
   extends Model[CrossValidatorModel] with CrossValidatorParams {
 
-  override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = {
+  override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = {
     bestModel.transform(dataset, paramMap)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java
----------------------------------------------------------------------
diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java
index 47f1f46..56a9dbd 100644
--- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java
+++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java
@@ -26,7 +26,7 @@ import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.mllib.regression.LabeledPoint;
 import org.apache.spark.ml.classification.LogisticRegression;
 import org.apache.spark.ml.feature.StandardScaler;
-import org.apache.spark.sql.SchemaRDD;
+import org.apache.spark.sql.DataFrame;
 import org.apache.spark.sql.SQLContext;
 import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList;
 
@@ -37,7 +37,7 @@ public class JavaPipelineSuite {
 
   private transient JavaSparkContext jsc;
   private transient SQLContext jsql;
-  private transient SchemaRDD dataset;
+  private transient DataFrame dataset;
 
   @Before
   public void setUp() {
@@ -65,7 +65,7 @@ public class JavaPipelineSuite {
       .setStages(new PipelineStage[] {scaler, lr});
     PipelineModel model = pipeline.fit(dataset);
     model.transform(dataset).registerTempTable("prediction");
-    SchemaRDD predictions = jsql.sql("SELECT label, score, prediction FROM prediction");
+    DataFrame predictions = jsql.sql("SELECT label, score, prediction FROM prediction");
     predictions.collectAsList();
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java
----------------------------------------------------------------------
diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java
index 2eba833..f4ba23c 100644
--- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java
+++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java
@@ -26,7 +26,7 @@ import org.junit.Test;
 
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.mllib.regression.LabeledPoint;
-import org.apache.spark.sql.SchemaRDD;
+import org.apache.spark.sql.DataFrame;
 import org.apache.spark.sql.SQLContext;
 import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList;
 
@@ -34,7 +34,7 @@ public class JavaLogisticRegressionSuite implements Serializable {
 
   private transient JavaSparkContext jsc;
   private transient SQLContext jsql;
-  private transient SchemaRDD dataset;
+  private transient DataFrame dataset;
 
   @Before
   public void setUp() {
@@ -55,7 +55,7 @@ public class JavaLogisticRegressionSuite implements Serializable {
     LogisticRegression lr = new LogisticRegression();
     LogisticRegressionModel model = lr.fit(dataset);
     model.transform(dataset).registerTempTable("prediction");
-    SchemaRDD predictions = jsql.sql("SELECT label, score, prediction FROM prediction");
+    DataFrame predictions = jsql.sql("SELECT label, score, prediction FROM prediction");
     predictions.collectAsList();
   }
 
@@ -67,7 +67,7 @@ public class JavaLogisticRegressionSuite implements Serializable {
     LogisticRegressionModel model = lr.fit(dataset);
     model.transform(dataset, model.threshold().w(0.8)) // overwrite threshold
       .registerTempTable("prediction");
-    SchemaRDD predictions = jsql.sql("SELECT label, score, prediction FROM prediction");
+    DataFrame predictions = jsql.sql("SELECT label, score, prediction FROM prediction");
     predictions.collectAsList();
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java
----------------------------------------------------------------------
diff --git a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java
index a9f1c4a..074b58c 100644
--- a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java
+++ b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java
@@ -30,7 +30,7 @@ import org.apache.spark.ml.classification.LogisticRegression;
 import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator;
 import org.apache.spark.ml.param.ParamMap;
 import org.apache.spark.mllib.regression.LabeledPoint;
-import org.apache.spark.sql.SchemaRDD;
+import org.apache.spark.sql.DataFrame;
 import org.apache.spark.sql.SQLContext;
 import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList;
 
@@ -38,7 +38,7 @@ public class JavaCrossValidatorSuite implements Serializable {
 
   private transient JavaSparkContext jsc;
   private transient SQLContext jsql;
-  private transient SchemaRDD dataset;
+  private transient DataFrame dataset;
 
   @Before
   public void setUp() {

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala
----------------------------------------------------------------------
diff --git a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala
index 4515084..2f175fb 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala
@@ -23,7 +23,7 @@ import org.scalatest.FunSuite
 import org.scalatest.mock.MockitoSugar.mock
 
 import org.apache.spark.ml.param.ParamMap
-import org.apache.spark.sql.SchemaRDD
+import org.apache.spark.sql.DataFrame
 
 class PipelineSuite extends FunSuite {
 
@@ -36,11 +36,11 @@ class PipelineSuite extends FunSuite {
     val estimator2 = mock[Estimator[MyModel]]
     val model2 = mock[MyModel]
     val transformer3 = mock[Transformer]
-    val dataset0 = mock[SchemaRDD]
-    val dataset1 = mock[SchemaRDD]
-    val dataset2 = mock[SchemaRDD]
-    val dataset3 = mock[SchemaRDD]
-    val dataset4 = mock[SchemaRDD]
+    val dataset0 = mock[DataFrame]
+    val dataset1 = mock[DataFrame]
+    val dataset2 = mock[DataFrame]
+    val dataset3 = mock[DataFrame]
+    val dataset4 = mock[DataFrame]
 
     when(estimator0.fit(meq(dataset0), any[ParamMap]())).thenReturn(model0)
     when(model0.transform(meq(dataset0), any[ParamMap]())).thenReturn(dataset1)
@@ -74,7 +74,7 @@ class PipelineSuite extends FunSuite {
     val estimator = mock[Estimator[MyModel]]
     val pipeline = new Pipeline()
       .setStages(Array(estimator, estimator))
-    val dataset = mock[SchemaRDD]
+    val dataset = mock[DataFrame]
     intercept[IllegalArgumentException] {
       pipeline.fit(dataset)
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
----------------------------------------------------------------------
diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
index e8030fe..1912afc 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
@@ -21,12 +21,12 @@ import org.scalatest.FunSuite
 
 import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput
 import org.apache.spark.mllib.util.MLlibTestSparkContext
-import org.apache.spark.sql.{SQLContext, SchemaRDD}
+import org.apache.spark.sql.{SQLContext, DataFrame}
 
 class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext {
 
   @transient var sqlContext: SQLContext = _
-  @transient var dataset: SchemaRDD = _
+  @transient var dataset: DataFrame = _
 
   override def beforeAll(): Unit = {
     super.beforeAll()
@@ -36,34 +36,28 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext {
   }
 
   test("logistic regression") {
-    val sqlContext = this.sqlContext
-    import sqlContext._
     val lr = new LogisticRegression
     val model = lr.fit(dataset)
     model.transform(dataset)
-      .select('label, 'prediction)
+      .select("label", "prediction")
       .collect()
   }
 
   test("logistic regression with setters") {
-    val sqlContext = this.sqlContext
-    import sqlContext._
     val lr = new LogisticRegression()
       .setMaxIter(10)
       .setRegParam(1.0)
     val model = lr.fit(dataset)
     model.transform(dataset, model.threshold -> 0.8) // overwrite threshold
-      .select('label, 'score, 'prediction)
+      .select("label", "score", "prediction")
       .collect()
   }
 
   test("logistic regression fit and transform with varargs") {
-    val sqlContext = this.sqlContext
-    import sqlContext._
     val lr = new LogisticRegression
     val model = lr.fit(dataset, lr.maxIter -> 10, lr.regParam -> 1.0)
     model.transform(dataset, model.threshold -> 0.8, model.scoreCol -> "probability")
-      .select('label, 'probability, 'prediction)
+      .select("label", "probability", "prediction")
       .collect()
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala
----------------------------------------------------------------------
diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala
index cdd4db1..58289ac 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala
@@ -350,7 +350,7 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging {
       numItemBlocks: Int = 3,
       targetRMSE: Double = 0.05): Unit = {
     val sqlContext = this.sqlContext
-    import sqlContext.{createSchemaRDD, symbolToUnresolvedAttribute}
+    import sqlContext.createSchemaRDD
     val als = new ALS()
       .setRank(rank)
       .setRegParam(regParam)
@@ -360,7 +360,7 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging {
     val alpha = als.getAlpha
     val model = als.fit(training)
     val predictions = model.transform(test)
-      .select('rating, 'prediction)
+      .select("rating", "prediction")
       .map { case Row(rating: Float, prediction: Float) =>
         (rating.toDouble, prediction.toDouble)
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala
----------------------------------------------------------------------
diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala
index 41cc13d..74104fa 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala
@@ -23,11 +23,11 @@ import org.apache.spark.ml.classification.LogisticRegression
 import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator
 import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput
 import org.apache.spark.mllib.util.MLlibTestSparkContext
-import org.apache.spark.sql.{SQLContext, SchemaRDD}
+import org.apache.spark.sql.{SQLContext, DataFrame}
 
 class CrossValidatorSuite extends FunSuite with MLlibTestSparkContext {
 
-  @transient var dataset: SchemaRDD = _
+  @transient var dataset: DataFrame = _
 
   override def beforeAll(): Unit = {
     super.beforeAll()

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/project/MimaExcludes.scala
----------------------------------------------------------------------
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index af0b0eb..e750fed 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -95,7 +95,20 @@ object MimaExcludes {
           ) ++ Seq(
             // SPARK-5166 Spark SQL API stabilization
             ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Transformer.transform"),
-            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Estimator.fit")
+            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Estimator.fit"),
+            ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.Transformer.transform"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Pipeline.fit"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PipelineModel.transform"),
+            ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.Estimator.fit"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Evaluator.evaluate"),
+            ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.Evaluator.evaluate"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.CrossValidator.fit"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.CrossValidatorModel.transform"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScaler.fit"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.transform"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.transform"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegression.fit"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.BinaryClassificationEvaluator.evaluate")
           ) ++ Seq(
             // SPARK-5270
             ProblemFilters.exclude[MissingMethodProblem](

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/python/pyspark/java_gateway.py
----------------------------------------------------------------------
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index a975dc1..a0a0284 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -111,10 +111,9 @@ def launch_gateway():
     java_import(gateway.jvm, "org.apache.spark.api.java.*")
     java_import(gateway.jvm, "org.apache.spark.api.python.*")
     java_import(gateway.jvm, "org.apache.spark.mllib.api.python.*")
-    java_import(gateway.jvm, "org.apache.spark.sql.SQLContext")
-    java_import(gateway.jvm, "org.apache.spark.sql.hive.HiveContext")
-    java_import(gateway.jvm, "org.apache.spark.sql.hive.LocalHiveContext")
-    java_import(gateway.jvm, "org.apache.spark.sql.hive.TestHiveContext")
+    # TODO(davies): move into sql
+    java_import(gateway.jvm, "org.apache.spark.sql.*")
+    java_import(gateway.jvm, "org.apache.spark.sql.hive.*")
     java_import(gateway.jvm, "scala.Tuple2")
 
     return gateway


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


[4/5] spark git commit: [SPARK-5097][SQL] DataFrame

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/python/pyspark/sql.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py
index 1990323..7d7550c 100644
--- a/python/pyspark/sql.py
+++ b/python/pyspark/sql.py
@@ -20,15 +20,19 @@ public classes of Spark SQL:
 
     - L{SQLContext}
       Main entry point for SQL functionality.
-    - L{SchemaRDD}
+    - L{DataFrame}
       A Resilient Distributed Dataset (RDD) with Schema information for the data contained. In
-      addition to normal RDD operations, SchemaRDDs also support SQL.
+      addition to normal RDD operations, DataFrames also support SQL.
+    - L{GroupedDataFrame}
+    - L{Column}
+      Column is a DataFrame with a single column.
     - L{Row}
       A Row of data returned by a Spark SQL query.
     - L{HiveContext}
       Main entry point for accessing data stored in Apache Hive..
 """
 
+import sys
 import itertools
 import decimal
 import datetime
@@ -36,6 +40,9 @@ import keyword
 import warnings
 import json
 import re
+import random
+import os
+from tempfile import NamedTemporaryFile
 from array import array
 from operator import itemgetter
 from itertools import imap
@@ -43,6 +50,7 @@ from itertools import imap
 from py4j.protocol import Py4JError
 from py4j.java_collections import ListConverter, MapConverter
 
+from pyspark.context import SparkContext
 from pyspark.rdd import RDD
 from pyspark.serializers import BatchedSerializer, AutoBatchedSerializer, PickleSerializer, \
     CloudPickleSerializer, UTF8Deserializer
@@ -54,7 +62,8 @@ __all__ = [
     "StringType", "BinaryType", "BooleanType", "DateType", "TimestampType", "DecimalType",
     "DoubleType", "FloatType", "ByteType", "IntegerType", "LongType",
     "ShortType", "ArrayType", "MapType", "StructField", "StructType",
-    "SQLContext", "HiveContext", "SchemaRDD", "Row"]
+    "SQLContext", "HiveContext", "DataFrame", "GroupedDataFrame", "Column", "Row",
+    "SchemaRDD"]
 
 
 class DataType(object):
@@ -1171,7 +1180,7 @@ def _create_cls(dataType):
 
     class Row(tuple):
 
-        """ Row in SchemaRDD """
+        """ Row in DataFrame """
         __DATATYPE__ = dataType
         __FIELDS__ = tuple(f.name for f in dataType.fields)
         __slots__ = ()
@@ -1198,7 +1207,7 @@ class SQLContext(object):
 
     """Main entry point for Spark SQL functionality.
 
-    A SQLContext can be used create L{SchemaRDD}, register L{SchemaRDD} as
+    A SQLContext can be used create L{DataFrame}, register L{DataFrame} as
     tables, execute SQL over tables, cache tables, and read parquet files.
     """
 
@@ -1209,8 +1218,8 @@ class SQLContext(object):
         :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instatiate a new
         SQLContext in the JVM, instead we make all calls to this object.
 
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> sqlCtx.inferSchema(srdd) # doctest: +IGNORE_EXCEPTION_DETAIL
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> sqlCtx.inferSchema(df) # doctest: +IGNORE_EXCEPTION_DETAIL
         Traceback (most recent call last):
             ...
         TypeError:...
@@ -1225,12 +1234,12 @@ class SQLContext(object):
         >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L,
         ...     b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1),
         ...     time=datetime(2014, 8, 1, 14, 1, 5))])
-        >>> srdd = sqlCtx.inferSchema(allTypes)
-        >>> srdd.registerTempTable("allTypes")
+        >>> df = sqlCtx.inferSchema(allTypes)
+        >>> df.registerTempTable("allTypes")
         >>> sqlCtx.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a '
         ...            'from allTypes where b and i > 0').collect()
         [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0...8, 1, 14, 1, 5), a=1)]
-        >>> srdd.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time,
+        >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time,
         ...                     x.row.a, x.list)).collect()
         [(1, u'string', 1.0, 1, True, ...(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])]
         """
@@ -1309,23 +1318,23 @@ class SQLContext(object):
         ...     [Row(field1=1, field2="row1"),
         ...      Row(field1=2, field2="row2"),
         ...      Row(field1=3, field2="row3")])
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> srdd.collect()[0]
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> df.collect()[0]
         Row(field1=1, field2=u'row1')
 
         >>> NestedRow = Row("f1", "f2")
         >>> nestedRdd1 = sc.parallelize([
         ...     NestedRow(array('i', [1, 2]), {"row1": 1.0}),
         ...     NestedRow(array('i', [2, 3]), {"row2": 2.0})])
-        >>> srdd = sqlCtx.inferSchema(nestedRdd1)
-        >>> srdd.collect()
+        >>> df = sqlCtx.inferSchema(nestedRdd1)
+        >>> df.collect()
         [Row(f1=[1, 2], f2={u'row1': 1.0}), ..., f2={u'row2': 2.0})]
 
         >>> nestedRdd2 = sc.parallelize([
         ...     NestedRow([[1, 2], [2, 3]], [1, 2]),
         ...     NestedRow([[2, 3], [3, 4]], [2, 3])])
-        >>> srdd = sqlCtx.inferSchema(nestedRdd2)
-        >>> srdd.collect()
+        >>> df = sqlCtx.inferSchema(nestedRdd2)
+        >>> df.collect()
         [Row(f1=[[1, 2], [2, 3]], f2=[1, 2]), ..., f2=[2, 3])]
 
         >>> from collections import namedtuple
@@ -1334,13 +1343,13 @@ class SQLContext(object):
         ...     [CustomRow(field1=1, field2="row1"),
         ...      CustomRow(field1=2, field2="row2"),
         ...      CustomRow(field1=3, field2="row3")])
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> srdd.collect()[0]
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> df.collect()[0]
         Row(field1=1, field2=u'row1')
         """
 
-        if isinstance(rdd, SchemaRDD):
-            raise TypeError("Cannot apply schema to SchemaRDD")
+        if isinstance(rdd, DataFrame):
+            raise TypeError("Cannot apply schema to DataFrame")
 
         first = rdd.first()
         if not first:
@@ -1384,10 +1393,10 @@ class SQLContext(object):
         >>> rdd2 = sc.parallelize([(1, "row1"), (2, "row2"), (3, "row3")])
         >>> schema = StructType([StructField("field1", IntegerType(), False),
         ...     StructField("field2", StringType(), False)])
-        >>> srdd = sqlCtx.applySchema(rdd2, schema)
-        >>> sqlCtx.registerRDDAsTable(srdd, "table1")
-        >>> srdd2 = sqlCtx.sql("SELECT * from table1")
-        >>> srdd2.collect()
+        >>> df = sqlCtx.applySchema(rdd2, schema)
+        >>> sqlCtx.registerRDDAsTable(df, "table1")
+        >>> df2 = sqlCtx.sql("SELECT * from table1")
+        >>> df2.collect()
         [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')]
 
         >>> from datetime import date, datetime
@@ -1410,15 +1419,15 @@ class SQLContext(object):
         ...         StructType([StructField("b", ShortType(), False)]), False),
         ...     StructField("list", ArrayType(ByteType(), False), False),
         ...     StructField("null", DoubleType(), True)])
-        >>> srdd = sqlCtx.applySchema(rdd, schema)
-        >>> results = srdd.map(
+        >>> df = sqlCtx.applySchema(rdd, schema)
+        >>> results = df.map(
         ...     lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int, x.float, x.date,
         ...         x.time, x.map["a"], x.struct.b, x.list, x.null))
         >>> results.collect()[0] # doctest: +NORMALIZE_WHITESPACE
         (127, -128, -32768, 32767, 2147483647, 1.0, datetime.date(2010, 1, 1),
              datetime.datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None)
 
-        >>> srdd.registerTempTable("table2")
+        >>> df.registerTempTable("table2")
         >>> sqlCtx.sql(
         ...   "SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " +
         ...     "short1 + 1 AS short1, short2 - 1 AS short2, int - 1 AS int, " +
@@ -1431,13 +1440,13 @@ class SQLContext(object):
         >>> abstract = "byte short float time map{} struct(b) list[]"
         >>> schema = _parse_schema_abstract(abstract)
         >>> typedSchema = _infer_schema_type(rdd.first(), schema)
-        >>> srdd = sqlCtx.applySchema(rdd, typedSchema)
-        >>> srdd.collect()
+        >>> df = sqlCtx.applySchema(rdd, typedSchema)
+        >>> df.collect()
         [Row(byte=127, short=-32768, float=1.0, time=..., list=[1, 2, 3])]
         """
 
-        if isinstance(rdd, SchemaRDD):
-            raise TypeError("Cannot apply schema to SchemaRDD")
+        if isinstance(rdd, DataFrame):
+            raise TypeError("Cannot apply schema to DataFrame")
 
         if not isinstance(schema, StructType):
             raise TypeError("schema should be StructType")
@@ -1457,8 +1466,8 @@ class SQLContext(object):
         rdd = rdd.map(converter)
 
         jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd())
-        srdd = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json())
-        return SchemaRDD(srdd, self)
+        df = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json())
+        return DataFrame(df, self)
 
     def registerRDDAsTable(self, rdd, tableName):
         """Registers the given RDD as a temporary table in the catalog.
@@ -1466,34 +1475,34 @@ class SQLContext(object):
         Temporary tables exist only during the lifetime of this instance of
         SQLContext.
 
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> sqlCtx.registerRDDAsTable(srdd, "table1")
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> sqlCtx.registerRDDAsTable(df, "table1")
         """
-        if (rdd.__class__ is SchemaRDD):
-            srdd = rdd._jschema_rdd.baseSchemaRDD()
-            self._ssql_ctx.registerRDDAsTable(srdd, tableName)
+        if (rdd.__class__ is DataFrame):
+            df = rdd._jdf
+            self._ssql_ctx.registerRDDAsTable(df, tableName)
         else:
-            raise ValueError("Can only register SchemaRDD as table")
+            raise ValueError("Can only register DataFrame as table")
 
     def parquetFile(self, path):
-        """Loads a Parquet file, returning the result as a L{SchemaRDD}.
+        """Loads a Parquet file, returning the result as a L{DataFrame}.
 
         >>> import tempfile, shutil
         >>> parquetFile = tempfile.mkdtemp()
         >>> shutil.rmtree(parquetFile)
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> srdd.saveAsParquetFile(parquetFile)
-        >>> srdd2 = sqlCtx.parquetFile(parquetFile)
-        >>> sorted(srdd.collect()) == sorted(srdd2.collect())
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> df.saveAsParquetFile(parquetFile)
+        >>> df2 = sqlCtx.parquetFile(parquetFile)
+        >>> sorted(df.collect()) == sorted(df2.collect())
         True
         """
-        jschema_rdd = self._ssql_ctx.parquetFile(path)
-        return SchemaRDD(jschema_rdd, self)
+        jdf = self._ssql_ctx.parquetFile(path)
+        return DataFrame(jdf, self)
 
     def jsonFile(self, path, schema=None, samplingRatio=1.0):
         """
         Loads a text file storing one JSON object per line as a
-        L{SchemaRDD}.
+        L{DataFrame}.
 
         If the schema is provided, applies the given schema to this
         JSON dataset.
@@ -1508,23 +1517,23 @@ class SQLContext(object):
         >>> for json in jsonStrings:
         ...   print>>ofn, json
         >>> ofn.close()
-        >>> srdd1 = sqlCtx.jsonFile(jsonFile)
-        >>> sqlCtx.registerRDDAsTable(srdd1, "table1")
-        >>> srdd2 = sqlCtx.sql(
+        >>> df1 = sqlCtx.jsonFile(jsonFile)
+        >>> sqlCtx.registerRDDAsTable(df1, "table1")
+        >>> df2 = sqlCtx.sql(
         ...   "SELECT field1 AS f1, field2 as f2, field3 as f3, "
         ...   "field6 as f4 from table1")
-        >>> for r in srdd2.collect():
+        >>> for r in df2.collect():
         ...     print r
         Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None)
         Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')])
         Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None)
 
-        >>> srdd3 = sqlCtx.jsonFile(jsonFile, srdd1.schema())
-        >>> sqlCtx.registerRDDAsTable(srdd3, "table2")
-        >>> srdd4 = sqlCtx.sql(
+        >>> df3 = sqlCtx.jsonFile(jsonFile, df1.schema())
+        >>> sqlCtx.registerRDDAsTable(df3, "table2")
+        >>> df4 = sqlCtx.sql(
         ...   "SELECT field1 AS f1, field2 as f2, field3 as f3, "
         ...   "field6 as f4 from table2")
-        >>> for r in srdd4.collect():
+        >>> for r in df4.collect():
         ...    print r
         Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None)
         Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')])
@@ -1536,23 +1545,23 @@ class SQLContext(object):
         ...         StructType([
         ...             StructField("field5",
         ...                 ArrayType(IntegerType(), False), True)]), False)])
-        >>> srdd5 = sqlCtx.jsonFile(jsonFile, schema)
-        >>> sqlCtx.registerRDDAsTable(srdd5, "table3")
-        >>> srdd6 = sqlCtx.sql(
+        >>> df5 = sqlCtx.jsonFile(jsonFile, schema)
+        >>> sqlCtx.registerRDDAsTable(df5, "table3")
+        >>> df6 = sqlCtx.sql(
         ...   "SELECT field2 AS f1, field3.field5 as f2, "
         ...   "field3.field5[0] as f3 from table3")
-        >>> srdd6.collect()
+        >>> df6.collect()
         [Row(f1=u'row1', f2=None, f3=None)...Row(f1=u'row3', f2=[], f3=None)]
         """
         if schema is None:
-            srdd = self._ssql_ctx.jsonFile(path, samplingRatio)
+            df = self._ssql_ctx.jsonFile(path, samplingRatio)
         else:
             scala_datatype = self._ssql_ctx.parseDataType(schema.json())
-            srdd = self._ssql_ctx.jsonFile(path, scala_datatype)
-        return SchemaRDD(srdd, self)
+            df = self._ssql_ctx.jsonFile(path, scala_datatype)
+        return DataFrame(df, self)
 
     def jsonRDD(self, rdd, schema=None, samplingRatio=1.0):
-        """Loads an RDD storing one JSON object per string as a L{SchemaRDD}.
+        """Loads an RDD storing one JSON object per string as a L{DataFrame}.
 
         If the schema is provided, applies the given schema to this
         JSON dataset.
@@ -1560,23 +1569,23 @@ class SQLContext(object):
         Otherwise, it samples the dataset with ratio `samplingRatio` to
         determine the schema.
 
-        >>> srdd1 = sqlCtx.jsonRDD(json)
-        >>> sqlCtx.registerRDDAsTable(srdd1, "table1")
-        >>> srdd2 = sqlCtx.sql(
+        >>> df1 = sqlCtx.jsonRDD(json)
+        >>> sqlCtx.registerRDDAsTable(df1, "table1")
+        >>> df2 = sqlCtx.sql(
         ...   "SELECT field1 AS f1, field2 as f2, field3 as f3, "
         ...   "field6 as f4 from table1")
-        >>> for r in srdd2.collect():
+        >>> for r in df2.collect():
         ...     print r
         Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None)
         Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')])
         Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None)
 
-        >>> srdd3 = sqlCtx.jsonRDD(json, srdd1.schema())
-        >>> sqlCtx.registerRDDAsTable(srdd3, "table2")
-        >>> srdd4 = sqlCtx.sql(
+        >>> df3 = sqlCtx.jsonRDD(json, df1.schema())
+        >>> sqlCtx.registerRDDAsTable(df3, "table2")
+        >>> df4 = sqlCtx.sql(
         ...   "SELECT field1 AS f1, field2 as f2, field3 as f3, "
         ...   "field6 as f4 from table2")
-        >>> for r in srdd4.collect():
+        >>> for r in df4.collect():
         ...     print r
         Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None)
         Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')])
@@ -1588,12 +1597,12 @@ class SQLContext(object):
         ...         StructType([
         ...             StructField("field5",
         ...                 ArrayType(IntegerType(), False), True)]), False)])
-        >>> srdd5 = sqlCtx.jsonRDD(json, schema)
-        >>> sqlCtx.registerRDDAsTable(srdd5, "table3")
-        >>> srdd6 = sqlCtx.sql(
+        >>> df5 = sqlCtx.jsonRDD(json, schema)
+        >>> sqlCtx.registerRDDAsTable(df5, "table3")
+        >>> df6 = sqlCtx.sql(
         ...   "SELECT field2 AS f1, field3.field5 as f2, "
         ...   "field3.field5[0] as f3 from table3")
-        >>> srdd6.collect()
+        >>> df6.collect()
         [Row(f1=u'row1', f2=None,...Row(f1=u'row3', f2=[], f3=None)]
 
         >>> sqlCtx.jsonRDD(sc.parallelize(['{}',
@@ -1615,33 +1624,33 @@ class SQLContext(object):
         keyed._bypass_serializer = True
         jrdd = keyed._jrdd.map(self._jvm.BytesToString())
         if schema is None:
-            srdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), samplingRatio)
+            df = self._ssql_ctx.jsonRDD(jrdd.rdd(), samplingRatio)
         else:
             scala_datatype = self._ssql_ctx.parseDataType(schema.json())
-            srdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype)
-        return SchemaRDD(srdd, self)
+            df = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype)
+        return DataFrame(df, self)
 
     def sql(self, sqlQuery):
-        """Return a L{SchemaRDD} representing the result of the given query.
+        """Return a L{DataFrame} representing the result of the given query.
 
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> sqlCtx.registerRDDAsTable(srdd, "table1")
-        >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1")
-        >>> srdd2.collect()
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> sqlCtx.registerRDDAsTable(df, "table1")
+        >>> df2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1")
+        >>> df2.collect()
         [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')]
         """
-        return SchemaRDD(self._ssql_ctx.sql(sqlQuery), self)
+        return DataFrame(self._ssql_ctx.sql(sqlQuery), self)
 
     def table(self, tableName):
-        """Returns the specified table as a L{SchemaRDD}.
+        """Returns the specified table as a L{DataFrame}.
 
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> sqlCtx.registerRDDAsTable(srdd, "table1")
-        >>> srdd2 = sqlCtx.table("table1")
-        >>> sorted(srdd.collect()) == sorted(srdd2.collect())
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> sqlCtx.registerRDDAsTable(df, "table1")
+        >>> df2 = sqlCtx.table("table1")
+        >>> sorted(df.collect()) == sorted(df2.collect())
         True
         """
-        return SchemaRDD(self._ssql_ctx.table(tableName), self)
+        return DataFrame(self._ssql_ctx.table(tableName), self)
 
     def cacheTable(self, tableName):
         """Caches the specified table in-memory."""
@@ -1707,7 +1716,7 @@ def _create_row(fields, values):
 class Row(tuple):
 
     """
-    A row in L{SchemaRDD}. The fields in it can be accessed like attributes.
+    A row in L{DataFrame}. The fields in it can be accessed like attributes.
 
     Row can be used to create a row object by using named arguments,
     the fields will be sorted by names.
@@ -1799,111 +1808,119 @@ def inherit_doc(cls):
     return cls
 
 
-@inherit_doc
-class SchemaRDD(RDD):
+class DataFrame(object):
 
-    """An RDD of L{Row} objects that has an associated schema.
+    """A collection of rows that have the same columns.
 
-    The underlying JVM object is a SchemaRDD, not a PythonRDD, so we can
-    utilize the relational query api exposed by Spark SQL.
+    A :class:`DataFrame` is equivalent to a relational table in Spark SQL,
+    and can be created using various functions in :class:`SQLContext`::
 
-    For normal L{pyspark.rdd.RDD} operations (map, count, etc.) the
-    L{SchemaRDD} is not operated on directly, as it's underlying
-    implementation is an RDD composed of Java objects. Instead it is
-    converted to a PythonRDD in the JVM, on which Python operations can
-    be done.
+        people = sqlContext.parquetFile("...")
 
-    This class receives raw tuples from Java but assigns a class to it in
-    all its data-collection methods (mapPartitionsWithIndex, collect, take,
-    etc) so that PySpark sees them as Row objects with named fields.
+    Once created, it can be manipulated using the various domain-specific-language
+    (DSL) functions defined in: [[DataFrame]], [[Column]].
+
+    To select a column from the data frame, use the apply method::
+
+        ageCol = people.age
+
+    Note that the :class:`Column` type can also be manipulated
+    through its various functions::
+
+        # The following creates a new column that increases everybody's age by 10.
+        people.age + 10
+
+
+    A more concrete example::
+
+        # To create DataFrame using SQLContext
+        people = sqlContext.parquetFile("...")
+        department = sqlContext.parquetFile("...")
+
+        people.filter(people.age > 30).join(department, people.deptId == department.id)) \
+          .groupBy(department.name, "gender").agg({"salary": "avg", "age": "max"})
     """
 
-    def __init__(self, jschema_rdd, sql_ctx):
+    def __init__(self, jdf, sql_ctx):
+        self._jdf = jdf
         self.sql_ctx = sql_ctx
-        self._sc = sql_ctx._sc
-        clsName = jschema_rdd.getClass().getName()
-        assert clsName.endswith("SchemaRDD"), "jschema_rdd must be SchemaRDD"
-        self._jschema_rdd = jschema_rdd
-        self._id = None
+        self._sc = sql_ctx and sql_ctx._sc
         self.is_cached = False
-        self.is_checkpointed = False
-        self.ctx = self.sql_ctx._sc
-        # the _jrdd is created by javaToPython(), serialized by pickle
-        self._jrdd_deserializer = AutoBatchedSerializer(PickleSerializer())
 
     @property
-    def _jrdd(self):
-        """Lazy evaluation of PythonRDD object.
+    def rdd(self):
+        """Return the content of the :class:`DataFrame` as an :class:`RDD`
+        of :class:`Row`s. """
+        if not hasattr(self, '_lazy_rdd'):
+            jrdd = self._jdf.javaToPython()
+            rdd = RDD(jrdd, self.sql_ctx._sc, BatchedSerializer(PickleSerializer()))
+            schema = self.schema()
 
-        Only done when a user calls methods defined by the
-        L{pyspark.rdd.RDD} super class (map, filter, etc.).
-        """
-        if not hasattr(self, '_lazy_jrdd'):
-            self._lazy_jrdd = self._jschema_rdd.baseSchemaRDD().javaToPython()
-        return self._lazy_jrdd
+            def applySchema(it):
+                cls = _create_cls(schema)
+                return itertools.imap(cls, it)
 
-    def id(self):
-        if self._id is None:
-            self._id = self._jrdd.id()
-        return self._id
+            self._lazy_rdd = rdd.mapPartitions(applySchema)
+
+        return self._lazy_rdd
 
     def limit(self, num):
         """Limit the result count to the number specified.
 
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> srdd.limit(2).collect()
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> df.limit(2).collect()
         [Row(field1=1, field2=u'row1'), Row(field1=2, field2=u'row2')]
-        >>> srdd.limit(0).collect()
+        >>> df.limit(0).collect()
         []
         """
-        rdd = self._jschema_rdd.baseSchemaRDD().limit(num)
-        return SchemaRDD(rdd, self.sql_ctx)
+        jdf = self._jdf.limit(num)
+        return DataFrame(jdf, self.sql_ctx)
 
     def toJSON(self, use_unicode=False):
-        """Convert a SchemaRDD into a MappedRDD of JSON documents; one document per row.
+        """Convert a DataFrame into a MappedRDD of JSON documents; one document per row.
 
-        >>> srdd1 = sqlCtx.jsonRDD(json)
-        >>> sqlCtx.registerRDDAsTable(srdd1, "table1")
-        >>> srdd2 = sqlCtx.sql( "SELECT * from table1")
-        >>> srdd2.toJSON().take(1)[0] == '{"field1":1,"field2":"row1","field3":{"field4":11}}'
+        >>> df1 = sqlCtx.jsonRDD(json)
+        >>> sqlCtx.registerRDDAsTable(df1, "table1")
+        >>> df2 = sqlCtx.sql( "SELECT * from table1")
+        >>> df2.toJSON().take(1)[0] == '{"field1":1,"field2":"row1","field3":{"field4":11}}'
         True
-        >>> srdd3 = sqlCtx.sql( "SELECT field3.field4 from table1")
-        >>> srdd3.toJSON().collect() == ['{"field4":11}', '{"field4":22}', '{"field4":33}']
+        >>> df3 = sqlCtx.sql( "SELECT field3.field4 from table1")
+        >>> df3.toJSON().collect() == ['{"field4":11}', '{"field4":22}', '{"field4":33}']
         True
         """
-        rdd = self._jschema_rdd.baseSchemaRDD().toJSON()
+        rdd = self._jdf.toJSON()
         return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode))
 
     def saveAsParquetFile(self, path):
         """Save the contents as a Parquet file, preserving the schema.
 
         Files that are written out using this method can be read back in as
-        a SchemaRDD using the L{SQLContext.parquetFile} method.
+        a DataFrame using the L{SQLContext.parquetFile} method.
 
         >>> import tempfile, shutil
         >>> parquetFile = tempfile.mkdtemp()
         >>> shutil.rmtree(parquetFile)
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> srdd.saveAsParquetFile(parquetFile)
-        >>> srdd2 = sqlCtx.parquetFile(parquetFile)
-        >>> sorted(srdd2.collect()) == sorted(srdd.collect())
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> df.saveAsParquetFile(parquetFile)
+        >>> df2 = sqlCtx.parquetFile(parquetFile)
+        >>> sorted(df2.collect()) == sorted(df.collect())
         True
         """
-        self._jschema_rdd.saveAsParquetFile(path)
+        self._jdf.saveAsParquetFile(path)
 
     def registerTempTable(self, name):
         """Registers this RDD as a temporary table using the given name.
 
         The lifetime of this temporary table is tied to the L{SQLContext}
-        that was used to create this SchemaRDD.
+        that was used to create this DataFrame.
 
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> srdd.registerTempTable("test")
-        >>> srdd2 = sqlCtx.sql("select * from test")
-        >>> sorted(srdd.collect()) == sorted(srdd2.collect())
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> df.registerTempTable("test")
+        >>> df2 = sqlCtx.sql("select * from test")
+        >>> sorted(df.collect()) == sorted(df2.collect())
         True
         """
-        self._jschema_rdd.registerTempTable(name)
+        self._jdf.registerTempTable(name)
 
     def registerAsTable(self, name):
         """DEPRECATED: use registerTempTable() instead"""
@@ -1911,62 +1928,61 @@ class SchemaRDD(RDD):
         self.registerTempTable(name)
 
     def insertInto(self, tableName, overwrite=False):
-        """Inserts the contents of this SchemaRDD into the specified table.
+        """Inserts the contents of this DataFrame into the specified table.
 
         Optionally overwriting any existing data.
         """
-        self._jschema_rdd.insertInto(tableName, overwrite)
+        self._jdf.insertInto(tableName, overwrite)
 
     def saveAsTable(self, tableName):
-        """Creates a new table with the contents of this SchemaRDD."""
-        self._jschema_rdd.saveAsTable(tableName)
+        """Creates a new table with the contents of this DataFrame."""
+        self._jdf.saveAsTable(tableName)
 
     def schema(self):
-        """Returns the schema of this SchemaRDD (represented by
+        """Returns the schema of this DataFrame (represented by
         a L{StructType})."""
-        return _parse_datatype_json_string(self._jschema_rdd.baseSchemaRDD().schema().json())
-
-    def schemaString(self):
-        """Returns the output schema in the tree format."""
-        return self._jschema_rdd.schemaString()
+        return _parse_datatype_json_string(self._jdf.schema().json())
 
     def printSchema(self):
         """Prints out the schema in the tree format."""
-        print self.schemaString()
+        print (self._jdf.schema().treeString())
 
     def count(self):
         """Return the number of elements in this RDD.
 
         Unlike the base RDD implementation of count, this implementation
-        leverages the query optimizer to compute the count on the SchemaRDD,
+        leverages the query optimizer to compute the count on the DataFrame,
         which supports features such as filter pushdown.
 
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> srdd.count()
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> df.count()
         3L
-        >>> srdd.count() == srdd.map(lambda x: x).count()
+        >>> df.count() == df.map(lambda x: x).count()
         True
         """
-        return self._jschema_rdd.count()
+        return self._jdf.count()
 
     def collect(self):
-        """Return a list that contains all of the rows in this RDD.
+        """Return a list that contains all of the rows.
 
         Each object in the list is a Row, the fields can be accessed as
         attributes.
 
-        Unlike the base RDD implementation of collect, this implementation
-        leverages the query optimizer to perform a collect on the SchemaRDD,
-        which supports features such as filter pushdown.
-
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> srdd.collect()
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> df.collect()
         [Row(field1=1, field2=u'row1'), ..., Row(field1=3, field2=u'row3')]
         """
-        with SCCallSiteSync(self.context) as css:
-            bytesInJava = self._jschema_rdd.baseSchemaRDD().collectToPython().iterator()
+        with SCCallSiteSync(self._sc) as css:
+            bytesInJava = self._jdf.javaToPython().collect().iterator()
         cls = _create_cls(self.schema())
-        return map(cls, self._collect_iterator_through_file(bytesInJava))
+        tempFile = NamedTemporaryFile(delete=False, dir=self._sc._temp_dir)
+        tempFile.close()
+        self._sc._writeToFile(bytesInJava, tempFile.name)
+        # Read the data into Python and deserialize it:
+        with open(tempFile.name, 'rb') as tempFile:
+            rs = list(BatchedSerializer(PickleSerializer()).load_stream(tempFile))
+        os.unlink(tempFile.name)
+        return [cls(r) for r in rs]
 
     def take(self, num):
         """Take the first num rows of the RDD.
@@ -1974,130 +1990,555 @@ class SchemaRDD(RDD):
         Each object in the list is a Row, the fields can be accessed as
         attributes.
 
-        Unlike the base RDD implementation of take, this implementation
-        leverages the query optimizer to perform a collect on a SchemaRDD,
-        which supports features such as filter pushdown.
-
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> srdd.take(2)
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> df.take(2)
         [Row(field1=1, field2=u'row1'), Row(field1=2, field2=u'row2')]
         """
         return self.limit(num).collect()
 
-    # Convert each object in the RDD to a Row with the right class
-    # for this SchemaRDD, so that fields can be accessed as attributes.
-    def mapPartitionsWithIndex(self, f, preservesPartitioning=False):
+    def map(self, f):
+        """ Return a new RDD by applying a function to each Row, it's a
+        shorthand for df.rdd.map()
         """
-        Return a new RDD by applying a function to each partition of this RDD,
-        while tracking the index of the original partition.
+        return self.rdd.map(f)
 
-        >>> rdd = sc.parallelize([1, 2, 3, 4], 4)
-        >>> def f(splitIndex, iterator): yield splitIndex
-        >>> rdd.mapPartitionsWithIndex(f).sum()
-        6
+    def mapPartitions(self, f, preservesPartitioning=False):
         """
-        rdd = RDD(self._jrdd, self._sc, self._jrdd_deserializer)
-
-        schema = self.schema()
+        Return a new RDD by applying a function to each partition.
 
-        def applySchema(_, it):
-            cls = _create_cls(schema)
-            return itertools.imap(cls, it)
-
-        objrdd = rdd.mapPartitionsWithIndex(applySchema, preservesPartitioning)
-        return objrdd.mapPartitionsWithIndex(f, preservesPartitioning)
+        >>> rdd = sc.parallelize([1, 2, 3, 4], 4)
+        >>> def f(iterator): yield 1
+        >>> rdd.mapPartitions(f).sum()
+        4
+        """
+        return self.rdd.mapPartitions(f, preservesPartitioning)
 
-    # We override the default cache/persist/checkpoint behavior
-    # as we want to cache the underlying SchemaRDD object in the JVM,
-    # not the PythonRDD checkpointed by the super class
     def cache(self):
+        """ Persist with the default storage level (C{MEMORY_ONLY_SER}).
+        """
         self.is_cached = True
-        self._jschema_rdd.cache()
+        self._jdf.cache()
         return self
 
     def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER):
+        """ Set the storage level to persist its values across operations
+        after the first time it is computed. This can only be used to assign
+        a new storage level if the RDD does not have a storage level set yet.
+        If no storage level is specified defaults to (C{MEMORY_ONLY_SER}).
+        """
         self.is_cached = True
-        javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel)
-        self._jschema_rdd.persist(javaStorageLevel)
+        javaStorageLevel = self._sc._getJavaStorageLevel(storageLevel)
+        self._jdf.persist(javaStorageLevel)
         return self
 
     def unpersist(self, blocking=True):
+        """ Mark it as non-persistent, and remove all blocks for it from
+        memory and disk.
+        """
         self.is_cached = False
-        self._jschema_rdd.unpersist(blocking)
+        self._jdf.unpersist(blocking)
         return self
 
-    def checkpoint(self):
-        self.is_checkpointed = True
-        self._jschema_rdd.checkpoint()
+    # def coalesce(self, numPartitions, shuffle=False):
+    #     rdd = self._jdf.coalesce(numPartitions, shuffle, None)
+    #     return DataFrame(rdd, self.sql_ctx)
 
-    def isCheckpointed(self):
-        return self._jschema_rdd.isCheckpointed()
+    def repartition(self, numPartitions):
+        """ Return a new :class:`DataFrame` that has exactly `numPartitions`
+        partitions.
+        """
+        rdd = self._jdf.repartition(numPartitions, None)
+        return DataFrame(rdd, self.sql_ctx)
 
-    def getCheckpointFile(self):
-        checkpointFile = self._jschema_rdd.getCheckpointFile()
-        if checkpointFile.isDefined():
-            return checkpointFile.get()
+    def sample(self, withReplacement, fraction, seed=None):
+        """
+        Return a sampled subset of this DataFrame.
 
-    def coalesce(self, numPartitions, shuffle=False):
-        rdd = self._jschema_rdd.coalesce(numPartitions, shuffle, None)
-        return SchemaRDD(rdd, self.sql_ctx)
+        >>> df = sqlCtx.inferSchema(rdd)
+        >>> df.sample(False, 0.5, 97).count()
+        2L
+        """
+        assert fraction >= 0.0, "Negative fraction value: %s" % fraction
+        seed = seed if seed is not None else random.randint(0, sys.maxint)
+        rdd = self._jdf.sample(withReplacement, fraction, long(seed))
+        return DataFrame(rdd, self.sql_ctx)
+
+    # def takeSample(self, withReplacement, num, seed=None):
+    #     """Return a fixed-size sampled subset of this DataFrame.
+    #
+    #     >>> df = sqlCtx.inferSchema(rdd)
+    #     >>> df.takeSample(False, 2, 97)
+    #     [Row(field1=3, field2=u'row3'), Row(field1=1, field2=u'row1')]
+    #     """
+    #     seed = seed if seed is not None else random.randint(0, sys.maxint)
+    #     with SCCallSiteSync(self.context) as css:
+    #         bytesInJava = self._jdf \
+    #             .takeSampleToPython(withReplacement, num, long(seed)) \
+    #             .iterator()
+    #     cls = _create_cls(self.schema())
+    #     return map(cls, self._collect_iterator_through_file(bytesInJava))
 
-    def distinct(self, numPartitions=None):
-        if numPartitions is None:
-            rdd = self._jschema_rdd.distinct()
+    @property
+    def dtypes(self):
+        """Return all column names and their data types as a list.
+        """
+        return [(f.name, str(f.dataType)) for f in self.schema().fields]
+
+    @property
+    def columns(self):
+        """ Return all column names as a list.
+        """
+        return [f.name for f in self.schema().fields]
+
+    def show(self):
+        raise NotImplemented
+
+    def join(self, other, joinExprs=None, joinType=None):
+        """
+        Join with another DataFrame, using the given join expression.
+        The following performs a full outer join between `df1` and `df2`::
+
+            df1.join(df2, df1.key == df2.key, "outer")
+
+        :param other: Right side of the join
+        :param joinExprs: Join expression
+        :param joinType: One of `inner`, `outer`, `left_outer`, `right_outer`,
+                         `semijoin`.
+        """
+        if joinType is None:
+            if joinExprs is None:
+                jdf = self._jdf.join(other._jdf)
+            else:
+                jdf = self._jdf.join(other._jdf, joinExprs)
         else:
-            rdd = self._jschema_rdd.distinct(numPartitions, None)
-        return SchemaRDD(rdd, self.sql_ctx)
+            jdf = self._jdf.join(other._jdf, joinExprs, joinType)
+        return DataFrame(jdf, self.sql_ctx)
+
+    def sort(self, *cols):
+        """ Return a new [[DataFrame]] sorted by the specified column,
+        in ascending column.
+
+        :param cols: The columns or expressions used for sorting
+        """
+        if not cols:
+            raise ValueError("should sort by at least one column")
+        for i, c in enumerate(cols):
+            if isinstance(c, basestring):
+                cols[i] = Column(c)
+        jcols = [c._jc for c in cols]
+        jdf = self._jdf.join(*jcols)
+        return DataFrame(jdf, self.sql_ctx)
+
+    sortBy = sort
+
+    def head(self, n=None):
+        """ Return the first `n` rows or the first row if n is None. """
+        if n is None:
+            rs = self.head(1)
+            return rs[0] if rs else None
+        return self.take(n)
+
+    def tail(self):
+        raise NotImplemented
+
+    def __getitem__(self, item):
+        if isinstance(item, basestring):
+            return Column(self._jdf.apply(item))
+
+        # TODO projection
+        raise IndexError
+
+    def __getattr__(self, name):
+        """ Return the column by given name """
+        if isinstance(name, basestring):
+            return Column(self._jdf.apply(name))
+        raise AttributeError
+
+    def As(self, name):
+        """ Alias the current DataFrame """
+        return DataFrame(getattr(self._jdf, "as")(name), self.sql_ctx)
+
+    def select(self, *cols):
+        """ Selecting a set of expressions.::
+
+            df.select()
+            df.select('colA', 'colB')
+            df.select(df.colA, df.colB + 1)
 
-    def intersection(self, other):
-        if (other.__class__ is SchemaRDD):
-            rdd = self._jschema_rdd.intersection(other._jschema_rdd)
-            return SchemaRDD(rdd, self.sql_ctx)
+        """
+        if not cols:
+            cols = ["*"]
+        if isinstance(cols[0], basestring):
+            cols = [_create_column_from_name(n) for n in cols]
         else:
-            raise ValueError("Can only intersect with another SchemaRDD")
+            cols = [c._jc for c in cols]
+        jcols = ListConverter().convert(cols, self._sc._gateway._gateway_client)
+        jdf = self._jdf.select(self._jdf.toColumnArray(jcols))
+        return DataFrame(jdf, self.sql_ctx)
 
-    def repartition(self, numPartitions):
-        rdd = self._jschema_rdd.repartition(numPartitions, None)
-        return SchemaRDD(rdd, self.sql_ctx)
+    def filter(self, condition):
+        """ Filtering rows using the given condition::
 
-    def subtract(self, other, numPartitions=None):
-        if (other.__class__ is SchemaRDD):
-            if numPartitions is None:
-                rdd = self._jschema_rdd.subtract(other._jschema_rdd)
-            else:
-                rdd = self._jschema_rdd.subtract(other._jschema_rdd,
-                                                 numPartitions)
-            return SchemaRDD(rdd, self.sql_ctx)
+            df.filter(df.age > 15)
+            df.where(df.age > 15)
+
+        """
+        return DataFrame(self._jdf.filter(condition._jc), self.sql_ctx)
+
+    where = filter
+
+    def groupBy(self, *cols):
+        """ Group the [[DataFrame]] using the specified columns,
+        so we can run aggregation on them. See :class:`GroupedDataFrame`
+        for all the available aggregate functions::
+
+            df.groupBy(df.department).avg()
+            df.groupBy("department", "gender").agg({
+                "salary": "avg",
+                "age":    "max",
+            })
+        """
+        if cols and isinstance(cols[0], basestring):
+            cols = [_create_column_from_name(n) for n in cols]
         else:
-            raise ValueError("Can only subtract another SchemaRDD")
+            cols = [c._jc for c in cols]
+        jcols = ListConverter().convert(cols, self._sc._gateway._gateway_client)
+        jdf = self._jdf.groupBy(self._jdf.toColumnArray(jcols))
+        return GroupedDataFrame(jdf, self.sql_ctx)
 
-    def sample(self, withReplacement, fraction, seed=None):
+    def agg(self, *exprs):
+        """ Aggregate on the entire [[DataFrame]] without groups
+        (shorthand for df.groupBy.agg())::
+
+            df.agg({"age": "max", "salary": "avg"})
         """
-        Return a sampled subset of this SchemaRDD.
+        return self.groupBy().agg(*exprs)
 
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> srdd.sample(False, 0.5, 97).count()
-        2L
+    def unionAll(self, other):
+        """ Return a new DataFrame containing union of rows in this
+        frame and another frame.
+
+        This is equivalent to `UNION ALL` in SQL.
         """
-        assert fraction >= 0.0, "Negative fraction value: %s" % fraction
-        seed = seed if seed is not None else random.randint(0, sys.maxint)
-        rdd = self._jschema_rdd.sample(withReplacement, fraction, long(seed))
-        return SchemaRDD(rdd, self.sql_ctx)
+        return DataFrame(self._jdf.unionAll(other._jdf), self.sql_ctx)
 
-    def takeSample(self, withReplacement, num, seed=None):
-        """Return a fixed-size sampled subset of this SchemaRDD.
+    def intersect(self, other):
+        """ Return a new [[DataFrame]] containing rows only in
+        both this frame and another frame.
 
-        >>> srdd = sqlCtx.inferSchema(rdd)
-        >>> srdd.takeSample(False, 2, 97)
-        [Row(field1=3, field2=u'row3'), Row(field1=1, field2=u'row1')]
+        This is equivalent to `INTERSECT` in SQL.
         """
-        seed = seed if seed is not None else random.randint(0, sys.maxint)
-        with SCCallSiteSync(self.context) as css:
-            bytesInJava = self._jschema_rdd.baseSchemaRDD() \
-                .takeSampleToPython(withReplacement, num, long(seed)) \
-                .iterator()
-        cls = _create_cls(self.schema())
-        return map(cls, self._collect_iterator_through_file(bytesInJava))
+        return DataFrame(self._jdf.intersect(other._jdf), self.sql_ctx)
+
+    def Except(self, other):
+        """ Return a new [[DataFrame]] containing rows in this frame
+        but not in another frame.
+
+        This is equivalent to `EXCEPT` in SQL.
+        """
+        return DataFrame(getattr(self._jdf, "except")(other._jdf), self.sql_ctx)
+
+    def sample(self, withReplacement, fraction, seed=None):
+        """ Return a new DataFrame by sampling a fraction of rows. """
+        if seed is None:
+            jdf = self._jdf.sample(withReplacement, fraction)
+        else:
+            jdf = self._jdf.sample(withReplacement, fraction, seed)
+        return DataFrame(jdf, self.sql_ctx)
+
+    def addColumn(self, colName, col):
+        """ Return a new [[DataFrame]] by adding a column. """
+        return self.select('*', col.As(colName))
+
+    def removeColumn(self, colName):
+        raise NotImplemented
+
+
+# Having SchemaRDD for backward compatibility (for docs)
+class SchemaRDD(DataFrame):
+    """
+    SchemaRDD is deprecated, please use DataFrame
+    """
+
+
+def dfapi(f):
+    def _api(self):
+        name = f.__name__
+        jdf = getattr(self._jdf, name)()
+        return DataFrame(jdf, self.sql_ctx)
+    _api.__name__ = f.__name__
+    _api.__doc__ = f.__doc__
+    return _api
+
+
+class GroupedDataFrame(object):
+
+    """
+    A set of methods for aggregations on a :class:`DataFrame`,
+    created by DataFrame.groupBy().
+    """
+
+    def __init__(self, jdf, sql_ctx):
+        self._jdf = jdf
+        self.sql_ctx = sql_ctx
+
+    def agg(self, *exprs):
+        """ Compute aggregates by specifying a map from column name
+        to aggregate methods.
+
+        The available aggregate methods are `avg`, `max`, `min`,
+        `sum`, `count`.
+
+        :param exprs: list or aggregate columns or a map from column
+                      name to agregate methods.
+        """
+        if len(exprs) == 1 and isinstance(exprs[0], dict):
+            jmap = MapConverter().convert(exprs[0],
+                                          self.sql_ctx._sc._gateway._gateway_client)
+            jdf = self._jdf.agg(jmap)
+        else:
+            # Columns
+            assert all(isinstance(c, Column) for c in exprs), "all exprs should be Columns"
+            jdf = self._jdf.agg(*exprs)
+        return DataFrame(jdf, self.sql_ctx)
+
+    @dfapi
+    def count(self):
+        """ Count the number of rows for each group. """
+
+    @dfapi
+    def mean(self):
+        """Compute the average value for each numeric columns
+        for each group. This is an alias for `avg`."""
+
+    @dfapi
+    def avg(self):
+        """Compute the average value for each numeric columns
+        for each group."""
+
+    @dfapi
+    def max(self):
+        """Compute the max value for each numeric columns for
+        each group. """
+
+    @dfapi
+    def min(self):
+        """Compute the min value for each numeric column for
+        each group."""
+
+    @dfapi
+    def sum(self):
+        """Compute the sum for each numeric columns for each
+        group."""
+
+
+SCALA_METHOD_MAPPINGS = {
+    '=': '$eq',
+    '>': '$greater',
+    '<': '$less',
+    '+': '$plus',
+    '-': '$minus',
+    '*': '$times',
+    '/': '$div',
+    '!': '$bang',
+    '@': '$at',
+    '#': '$hash',
+    '%': '$percent',
+    '^': '$up',
+    '&': '$amp',
+    '~': '$tilde',
+    '?': '$qmark',
+    '|': '$bar',
+    '\\': '$bslash',
+    ':': '$colon',
+}
+
+
+def _create_column_from_literal(literal):
+    sc = SparkContext._active_spark_context
+    return sc._jvm.Literal.apply(literal)
+
+
+def _create_column_from_name(name):
+    sc = SparkContext._active_spark_context
+    return sc._jvm.Column(name)
+
+
+def _scalaMethod(name):
+    """ Translate operators into methodName in Scala
+
+    For example:
+    >>> _scalaMethod('+')
+    '$plus'
+    >>> _scalaMethod('>=')
+    '$greater$eq'
+    >>> _scalaMethod('cast')
+    'cast'
+    """
+    return ''.join(SCALA_METHOD_MAPPINGS.get(c, c) for c in name)
+
+
+def _unary_op(name):
+    """ Create a method for given unary operator """
+    def _(self):
+        return Column(getattr(self._jc, _scalaMethod(name))(), self._jdf, self.sql_ctx)
+    return _
+
+
+def _bin_op(name):
+    """ Create a method for given binary operator """
+    def _(self, other):
+        if isinstance(other, Column):
+            jc = other._jc
+        else:
+            jc = _create_column_from_literal(other)
+        return Column(getattr(self._jc, _scalaMethod(name))(jc), self._jdf, self.sql_ctx)
+    return _
+
+
+def _reverse_op(name):
+    """ Create a method for binary operator (this object is on right side)
+    """
+    def _(self, other):
+        return Column(getattr(_create_column_from_literal(other), _scalaMethod(name))(self._jc),
+                      self._jdf, self.sql_ctx)
+    return _
+
+
+class Column(DataFrame):
+
+    """
+    A column in a DataFrame.
+
+    `Column` instances can be created by:
+    {{{
+    // 1. Select a column out of a DataFrame
+    df.colName
+    df["colName"]
+
+    // 2. Create from an expression
+    df["colName"] + 1
+    }}}
+    """
+
+    def __init__(self, jc, jdf=None, sql_ctx=None):
+        self._jc = jc
+        super(Column, self).__init__(jdf, sql_ctx)
+
+    # arithmetic operators
+    __neg__ = _unary_op("unary_-")
+    __add__ = _bin_op("+")
+    __sub__ = _bin_op("-")
+    __mul__ = _bin_op("*")
+    __div__ = _bin_op("/")
+    __mod__ = _bin_op("%")
+    __radd__ = _bin_op("+")
+    __rsub__ = _reverse_op("-")
+    __rmul__ = _bin_op("*")
+    __rdiv__ = _reverse_op("/")
+    __rmod__ = _reverse_op("%")
+    __abs__ = _unary_op("abs")
+    abs = _unary_op("abs")
+    sqrt = _unary_op("sqrt")
+
+    # logistic operators
+    __eq__ = _bin_op("===")
+    __ne__ = _bin_op("!==")
+    __lt__ = _bin_op("<")
+    __le__ = _bin_op("<=")
+    __ge__ = _bin_op(">=")
+    __gt__ = _bin_op(">")
+    # `and`, `or`, `not` cannot be overloaded in Python
+    And = _bin_op('&&')
+    Or = _bin_op('||')
+    Not = _unary_op('unary_!')
+
+    # bitwise operators
+    __and__ = _bin_op("&")
+    __or__ = _bin_op("|")
+    __invert__ = _unary_op("unary_~")
+    __xor__ = _bin_op("^")
+    # __lshift__ = _bin_op("<<")
+    # __rshift__ = _bin_op(">>")
+    __rand__ = _bin_op("&")
+    __ror__ = _bin_op("|")
+    __rxor__ = _bin_op("^")
+    # __rlshift__ = _reverse_op("<<")
+    # __rrshift__ = _reverse_op(">>")
+
+    # container operators
+    __contains__ = _bin_op("contains")
+    __getitem__ = _bin_op("getItem")
+    # __getattr__ = _bin_op("getField")
+
+    # string methods
+    rlike = _bin_op("rlike")
+    like = _bin_op("like")
+    startswith = _bin_op("startsWith")
+    endswith = _bin_op("endsWith")
+    upper = _unary_op("upper")
+    lower = _unary_op("lower")
+
+    def substr(self, startPos, pos):
+        if type(startPos) != type(pos):
+            raise TypeError("Can not mix the type")
+        if isinstance(startPos, (int, long)):
+
+            jc = self._jc.substr(startPos, pos)
+        elif isinstance(startPos, Column):
+            jc = self._jc.substr(startPos._jc, pos._jc)
+        else:
+            raise TypeError("Unexpected type: %s" % type(startPos))
+        return Column(jc, self._jdf, self.sql_ctx)
+
+    __getslice__ = substr
+
+    # order
+    asc = _unary_op("asc")
+    desc = _unary_op("desc")
+
+    isNull = _unary_op("isNull")
+    isNotNull = _unary_op("isNotNull")
+
+    # `as` is keyword
+    def As(self, alias):
+        return Column(getattr(self._jsc, "as")(alias), self._jdf, self.sql_ctx)
+
+    def cast(self, dataType):
+        if self.sql_ctx is None:
+            sc = SparkContext._active_spark_context
+            ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc())
+        else:
+            ssql_ctx = self.sql_ctx._ssql_ctx
+        jdt = ssql_ctx.parseDataType(dataType.json())
+        return Column(self._jc.cast(jdt), self._jdf, self.sql_ctx)
+
+
+def _aggregate_func(name):
+    """ Creat a function for aggregator by name"""
+    def _(col):
+        sc = SparkContext._active_spark_context
+        if isinstance(col, Column):
+            jcol = col._jc
+        else:
+            jcol = _create_column_from_name(col)
+        # FIXME: can not access dsl.min/max ...
+        jc = getattr(sc._jvm.org.apache.spark.sql.dsl(), name)(jcol)
+        return Column(jc)
+    return staticmethod(_)
+
+
+class Aggregator(object):
+    """
+    A collections of builtin aggregators
+    """
+    max = _aggregate_func("max")
+    min = _aggregate_func("min")
+    avg = mean = _aggregate_func("mean")
+    sum = _aggregate_func("sum")
+    first = _aggregate_func("first")
+    last = _aggregate_func("last")
+    count = _aggregate_func("count")
 
 
 def _test():

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/python/pyspark/tests.py
----------------------------------------------------------------------
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
index b474fcf..e8e207a 100644
--- a/python/pyspark/tests.py
+++ b/python/pyspark/tests.py
@@ -806,6 +806,9 @@ class SQLTests(ReusedPySparkTestCase):
 
     def setUp(self):
         self.sqlCtx = SQLContext(self.sc)
+        self.testData = [Row(key=i, value=str(i)) for i in range(100)]
+        rdd = self.sc.parallelize(self.testData)
+        self.df = self.sqlCtx.inferSchema(rdd)
 
     def test_udf(self):
         self.sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType())
@@ -821,7 +824,7 @@ class SQLTests(ReusedPySparkTestCase):
     def test_udf_with_array_type(self):
         d = [Row(l=range(3), d={"key": range(5)})]
         rdd = self.sc.parallelize(d)
-        srdd = self.sqlCtx.inferSchema(rdd).registerTempTable("test")
+        self.sqlCtx.inferSchema(rdd).registerTempTable("test")
         self.sqlCtx.registerFunction("copylist", lambda l: list(l), ArrayType(IntegerType()))
         self.sqlCtx.registerFunction("maplen", lambda d: len(d), IntegerType())
         [(l1, l2)] = self.sqlCtx.sql("select copylist(l), maplen(d) from test").collect()
@@ -839,68 +842,51 @@ class SQLTests(ReusedPySparkTestCase):
 
     def test_basic_functions(self):
         rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}'])
-        srdd = self.sqlCtx.jsonRDD(rdd)
-        srdd.count()
-        srdd.collect()
-        srdd.schemaString()
-        srdd.schema()
+        df = self.sqlCtx.jsonRDD(rdd)
+        df.count()
+        df.collect()
+        df.schema()
 
         # cache and checkpoint
-        self.assertFalse(srdd.is_cached)
-        srdd.persist()
-        srdd.unpersist()
-        srdd.cache()
-        self.assertTrue(srdd.is_cached)
-        self.assertFalse(srdd.isCheckpointed())
-        self.assertEqual(None, srdd.getCheckpointFile())
-
-        srdd = srdd.coalesce(2, True)
-        srdd = srdd.repartition(3)
-        srdd = srdd.distinct()
-        srdd.intersection(srdd)
-        self.assertEqual(2, srdd.count())
-
-        srdd.registerTempTable("temp")
-        srdd = self.sqlCtx.sql("select foo from temp")
-        srdd.count()
-        srdd.collect()
-
-    def test_distinct(self):
-        rdd = self.sc.parallelize(['{"a": 1}', '{"b": 2}', '{"c": 3}']*10, 10)
-        srdd = self.sqlCtx.jsonRDD(rdd)
-        self.assertEquals(srdd.getNumPartitions(), 10)
-        self.assertEquals(srdd.distinct().count(), 3)
-        result = srdd.distinct(5)
-        self.assertEquals(result.getNumPartitions(), 5)
-        self.assertEquals(result.count(), 3)
+        self.assertFalse(df.is_cached)
+        df.persist()
+        df.unpersist()
+        df.cache()
+        self.assertTrue(df.is_cached)
+        self.assertEqual(2, df.count())
+
+        df.registerTempTable("temp")
+        df = self.sqlCtx.sql("select foo from temp")
+        df.count()
+        df.collect()
 
     def test_apply_schema_to_row(self):
-        srdd = self.sqlCtx.jsonRDD(self.sc.parallelize(["""{"a":2}"""]))
-        srdd2 = self.sqlCtx.applySchema(srdd.map(lambda x: x), srdd.schema())
-        self.assertEqual(srdd.collect(), srdd2.collect())
+        df = self.sqlCtx.jsonRDD(self.sc.parallelize(["""{"a":2}"""]))
+        df2 = self.sqlCtx.applySchema(df.map(lambda x: x), df.schema())
+        self.assertEqual(df.collect(), df2.collect())
 
         rdd = self.sc.parallelize(range(10)).map(lambda x: Row(a=x))
-        srdd3 = self.sqlCtx.applySchema(rdd, srdd.schema())
-        self.assertEqual(10, srdd3.count())
+        df3 = self.sqlCtx.applySchema(rdd, df.schema())
+        self.assertEqual(10, df3.count())
 
     def test_serialize_nested_array_and_map(self):
         d = [Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")})]
         rdd = self.sc.parallelize(d)
-        srdd = self.sqlCtx.inferSchema(rdd)
-        row = srdd.first()
+        df = self.sqlCtx.inferSchema(rdd)
+        row = df.head()
         self.assertEqual(1, len(row.l))
         self.assertEqual(1, row.l[0].a)
         self.assertEqual("2", row.d["key"].d)
 
-        l = srdd.map(lambda x: x.l).first()
+        l = df.map(lambda x: x.l).first()
         self.assertEqual(1, len(l))
         self.assertEqual('s', l[0].b)
 
-        d = srdd.map(lambda x: x.d).first()
+        d = df.map(lambda x: x.d).first()
         self.assertEqual(1, len(d))
         self.assertEqual(1.0, d["key"].c)
 
-        row = srdd.map(lambda x: x.d["key"]).first()
+        row = df.map(lambda x: x.d["key"]).first()
         self.assertEqual(1.0, row.c)
         self.assertEqual("2", row.d)
 
@@ -908,26 +894,26 @@ class SQLTests(ReusedPySparkTestCase):
         d = [Row(l=[], d={}),
              Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}, s="")]
         rdd = self.sc.parallelize(d)
-        srdd = self.sqlCtx.inferSchema(rdd)
-        self.assertEqual([], srdd.map(lambda r: r.l).first())
-        self.assertEqual([None, ""], srdd.map(lambda r: r.s).collect())
-        srdd.registerTempTable("test")
+        df = self.sqlCtx.inferSchema(rdd)
+        self.assertEqual([], df.map(lambda r: r.l).first())
+        self.assertEqual([None, ""], df.map(lambda r: r.s).collect())
+        df.registerTempTable("test")
         result = self.sqlCtx.sql("SELECT l[0].a from test where d['key'].d = '2'")
-        self.assertEqual(1, result.first()[0])
+        self.assertEqual(1, result.head()[0])
 
-        srdd2 = self.sqlCtx.inferSchema(rdd, 1.0)
-        self.assertEqual(srdd.schema(), srdd2.schema())
-        self.assertEqual({}, srdd2.map(lambda r: r.d).first())
-        self.assertEqual([None, ""], srdd2.map(lambda r: r.s).collect())
-        srdd2.registerTempTable("test2")
+        df2 = self.sqlCtx.inferSchema(rdd, 1.0)
+        self.assertEqual(df.schema(), df2.schema())
+        self.assertEqual({}, df2.map(lambda r: r.d).first())
+        self.assertEqual([None, ""], df2.map(lambda r: r.s).collect())
+        df2.registerTempTable("test2")
         result = self.sqlCtx.sql("SELECT l[0].a from test2 where d['key'].d = '2'")
-        self.assertEqual(1, result.first()[0])
+        self.assertEqual(1, result.head()[0])
 
     def test_struct_in_map(self):
         d = [Row(m={Row(i=1): Row(s="")})]
         rdd = self.sc.parallelize(d)
-        srdd = self.sqlCtx.inferSchema(rdd)
-        k, v = srdd.first().m.items()[0]
+        df = self.sqlCtx.inferSchema(rdd)
+        k, v = df.head().m.items()[0]
         self.assertEqual(1, k.i)
         self.assertEqual("", v.s)
 
@@ -935,9 +921,9 @@ class SQLTests(ReusedPySparkTestCase):
         row = Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")})
         self.assertEqual(1, row.asDict()['l'][0].a)
         rdd = self.sc.parallelize([row])
-        srdd = self.sqlCtx.inferSchema(rdd)
-        srdd.registerTempTable("test")
-        row = self.sqlCtx.sql("select l, d from test").first()
+        df = self.sqlCtx.inferSchema(rdd)
+        df.registerTempTable("test")
+        row = self.sqlCtx.sql("select l, d from test").head()
         self.assertEqual(1, row.asDict()["l"][0].a)
         self.assertEqual(1.0, row.asDict()['d']['key'].c)
 
@@ -945,12 +931,12 @@ class SQLTests(ReusedPySparkTestCase):
         from pyspark.tests import ExamplePoint, ExamplePointUDT
         row = Row(label=1.0, point=ExamplePoint(1.0, 2.0))
         rdd = self.sc.parallelize([row])
-        srdd = self.sqlCtx.inferSchema(rdd)
-        schema = srdd.schema()
+        df = self.sqlCtx.inferSchema(rdd)
+        schema = df.schema()
         field = [f for f in schema.fields if f.name == "point"][0]
         self.assertEqual(type(field.dataType), ExamplePointUDT)
-        srdd.registerTempTable("labeled_point")
-        point = self.sqlCtx.sql("SELECT point FROM labeled_point").first().point
+        df.registerTempTable("labeled_point")
+        point = self.sqlCtx.sql("SELECT point FROM labeled_point").head().point
         self.assertEqual(point, ExamplePoint(1.0, 2.0))
 
     def test_apply_schema_with_udt(self):
@@ -959,21 +945,52 @@ class SQLTests(ReusedPySparkTestCase):
         rdd = self.sc.parallelize([row])
         schema = StructType([StructField("label", DoubleType(), False),
                              StructField("point", ExamplePointUDT(), False)])
-        srdd = self.sqlCtx.applySchema(rdd, schema)
-        point = srdd.first().point
+        df = self.sqlCtx.applySchema(rdd, schema)
+        point = df.head().point
         self.assertEquals(point, ExamplePoint(1.0, 2.0))
 
     def test_parquet_with_udt(self):
         from pyspark.tests import ExamplePoint
         row = Row(label=1.0, point=ExamplePoint(1.0, 2.0))
         rdd = self.sc.parallelize([row])
-        srdd0 = self.sqlCtx.inferSchema(rdd)
+        df0 = self.sqlCtx.inferSchema(rdd)
         output_dir = os.path.join(self.tempdir.name, "labeled_point")
-        srdd0.saveAsParquetFile(output_dir)
-        srdd1 = self.sqlCtx.parquetFile(output_dir)
-        point = srdd1.first().point
+        df0.saveAsParquetFile(output_dir)
+        df1 = self.sqlCtx.parquetFile(output_dir)
+        point = df1.head().point
         self.assertEquals(point, ExamplePoint(1.0, 2.0))
 
+    def test_column_operators(self):
+        from pyspark.sql import Column, LongType
+        ci = self.df.key
+        cs = self.df.value
+        c = ci == cs
+        self.assertTrue(isinstance((- ci - 1 - 2) % 3 * 2.5 / 3.5, Column))
+        rcc = (1 + ci), (1 - ci), (1 * ci), (1 / ci), (1 % ci)
+        self.assertTrue(all(isinstance(c, Column) for c in rcc))
+        cb = [ci == 5, ci != 0, ci > 3, ci < 4, ci >= 0, ci <= 7, ci and cs, ci or cs]
+        self.assertTrue(all(isinstance(c, Column) for c in cb))
+        cbit = (ci & ci), (ci | ci), (ci ^ ci), (~ci)
+        self.assertTrue(all(isinstance(c, Column) for c in cbit))
+        css = cs.like('a'), cs.rlike('a'), cs.asc(), cs.desc(), cs.startswith('a'), cs.endswith('a')
+        self.assertTrue(all(isinstance(c, Column) for c in css))
+        self.assertTrue(isinstance(ci.cast(LongType()), Column))
+
+    def test_column_select(self):
+        df = self.df
+        self.assertEqual(self.testData, df.select("*").collect())
+        self.assertEqual(self.testData, df.select(df.key, df.value).collect())
+        self.assertEqual([Row(value='1')], df.where(df.key == 1).select(df.value).collect())
+
+    def test_aggregator(self):
+        df = self.df
+        g = df.groupBy()
+        self.assertEqual([99, 100], sorted(g.agg({'key': 'max', 'value': 'count'}).collect()[0]))
+        self.assertEqual([Row(**{"AVG(key#0)": 49.5})], g.mean().collect())
+        # TODO(davies): fix aggregators
+        from pyspark.sql import Aggregator as Agg
+        # self.assertEqual((0, '100'), tuple(g.agg(Agg.first(df.key), Agg.last(df.value)).first()))
+
 
 class InputFormatTests(ReusedPySparkTestCase):
 

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
index 22941ed..4c5fb3f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
@@ -47,7 +47,7 @@ object NewRelationInstances extends Rule[LogicalPlan] {
       .toSet
 
     plan transform {
-      case l: MultiInstanceRelation if multiAppearance contains l => l.newInstance
+      case l: MultiInstanceRelation if multiAppearance.contains(l) => l.newInstance()
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
index 3035d93..f388cd5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
@@ -77,6 +77,9 @@ abstract class Attribute extends NamedExpression {
  * For example the SQL expression "1 + 1 AS a" could be represented as follows:
  *  Alias(Add(Literal(1), Literal(1), "a")()
  *
+ * Note that exprId and qualifiers are in a separate parameter list because
+ * we only pattern match on child and name.
+ *
  * @param child the computation being performed
  * @param name the name to be associated with the result of computing [[child]].
  * @param exprId A globally unique id used to check if an [[AttributeReference]] refers to this

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala
index 613f4bb..5dc0539 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala
@@ -17,9 +17,24 @@
 
 package org.apache.spark.sql.catalyst.plans
 
+object JoinType {
+  def apply(typ: String): JoinType = typ.toLowerCase.replace("_", "") match {
+    case "inner" => Inner
+    case "outer" | "full" | "fullouter" => FullOuter
+    case "leftouter" | "left" => LeftOuter
+    case "rightouter" | "right" => RightOuter
+    case "leftsemi" => LeftSemi
+  }
+}
+
 sealed abstract class JoinType
+
 case object Inner extends JoinType
+
 case object LeftOuter extends JoinType
+
 case object RightOuter extends JoinType
+
 case object FullOuter extends JoinType
+
 case object LeftSemi extends JoinType

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala
index 1976998..d90af45 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala
@@ -19,10 +19,14 @@ package org.apache.spark.sql.catalyst.plans.logical
 
 import org.apache.spark.sql.catalyst.analysis
 import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.types.{StructType, StructField}
 
 object LocalRelation {
-  def apply(output: Attribute*) =
-    new LocalRelation(output)
+  def apply(output: Attribute*): LocalRelation = new LocalRelation(output)
+
+  def apply(output1: StructField, output: StructField*): LocalRelation = new LocalRelation(
+    StructType(output1 +: output).toAttributes
+  )
 }
 
 case class LocalRelation(output: Seq[Attribute], data: Seq[Product] = Nil)

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala
index e715d94..bc22f68 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala
@@ -80,7 +80,7 @@ private[sql] trait CacheManager {
    * the in-memory columnar representation of the underlying table is expensive.
    */
   private[sql] def cacheQuery(
-      query: SchemaRDD,
+      query: DataFrame,
       tableName: Option[String] = None,
       storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = writeLock {
     val planToCache = query.queryExecution.analyzed
@@ -100,7 +100,7 @@ private[sql] trait CacheManager {
   }
 
   /** Removes the data for the given SchemaRDD from the cache */
-  private[sql] def uncacheQuery(query: SchemaRDD, blocking: Boolean = true): Unit = writeLock {
+  private[sql] def uncacheQuery(query: DataFrame, blocking: Boolean = true): Unit = writeLock {
     val planToCache = query.queryExecution.analyzed
     val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan))
     require(dataIndex >= 0, s"Table $query is not cached.")
@@ -110,7 +110,7 @@ private[sql] trait CacheManager {
 
   /** Tries to remove the data for the given SchemaRDD from the cache if it's cached */
   private[sql] def tryUncacheQuery(
-      query: SchemaRDD,
+      query: DataFrame,
       blocking: Boolean = true): Boolean = writeLock {
     val planToCache = query.queryExecution.analyzed
     val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan))
@@ -123,7 +123,7 @@ private[sql] trait CacheManager {
   }
 
   /** Optionally returns cached data for the given SchemaRDD */
-  private[sql] def lookupCachedData(query: SchemaRDD): Option[CachedData] = readLock {
+  private[sql] def lookupCachedData(query: DataFrame): Option[CachedData] = readLock {
     lookupCachedData(query.queryExecution.analyzed)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/119f45d6/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
new file mode 100644
index 0000000..7fc8347
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
@@ -0,0 +1,528 @@
+/*
+* 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
+
+import scala.language.implicitConversions
+
+import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, Star}
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.{Literal => LiteralExpr}
+import org.apache.spark.sql.catalyst.plans.logical.{Project, LogicalPlan}
+import org.apache.spark.sql.types._
+
+
+object Column {
+  def unapply(col: Column): Option[Expression] = Some(col.expr)
+
+  def apply(colName: String): Column = new Column(colName)
+}
+
+
+/**
+ * A column in a [[DataFrame]].
+ *
+ * `Column` instances can be created by:
+ * {{{
+ *   // 1. Select a column out of a DataFrame
+ *   df("colName")
+ *
+ *   // 2. Create a literal expression
+ *   Literal(1)
+ *
+ *   // 3. Create new columns from
+ * }}}
+ *
+ */
+// TODO: Improve documentation.
+class Column(
+    sqlContext: Option[SQLContext],
+    plan: Option[LogicalPlan],
+    val expr: Expression)
+  extends DataFrame(sqlContext, plan) with ExpressionApi {
+
+  /** Turn a Catalyst expression into a `Column`. */
+  protected[sql] def this(expr: Expression) = this(None, None, expr)
+
+  /**
+   * Create a new `Column` expression based on a column or attribute name.
+   * The resolution of this is the same as SQL. For example:
+   *
+   * - "colName" becomes an expression selecting the column named "colName".
+   * - "*" becomes an expression selecting all columns.
+   * - "df.*" becomes an expression selecting all columns in data frame "df".
+   */
+  def this(name: String) = this(name match {
+    case "*" => Star(None)
+    case _ if name.endsWith(".*") => Star(Some(name.substring(0, name.length - 2)))
+    case _ => UnresolvedAttribute(name)
+  })
+
+  override def isComputable: Boolean = sqlContext.isDefined && plan.isDefined
+
+  /**
+   * An implicit conversion function internal to this class. This function creates a new Column
+   * based on an expression. If the expression itself is not named, it aliases the expression
+   * by calling it "col".
+   */
+  private[this] implicit def toColumn(expr: Expression): Column = {
+    val projectedPlan = plan.map { p =>
+      Project(Seq(expr match {
+        case named: NamedExpression => named
+        case unnamed: Expression => Alias(unnamed, "col")()
+      }), p)
+    }
+    new Column(sqlContext, projectedPlan, expr)
+  }
+
+  /**
+   * Unary minus, i.e. negate the expression.
+   * {{{
+   *   // Select the amount column and negates all values.
+   *   df.select( -df("amount") )
+   * }}}
+   */
+  override def unary_- : Column = UnaryMinus(expr)
+
+  /**
+   * Bitwise NOT.
+   * {{{
+   *   // Select the flags column and negate every bit.
+   *   df.select( ~df("flags") )
+   * }}}
+   */
+  override def unary_~ : Column = BitwiseNot(expr)
+
+  /**
+   * Invert a boolean expression, i.e. NOT.
+   * {{
+   *   // Select rows that are not active (isActive === false)
+   *   df.select( !df("isActive") )
+   * }}
+   */
+  override def unary_! : Column = Not(expr)
+
+
+  /**
+   * Equality test with an expression.
+   * {{{
+   *   // The following two both select rows in which colA equals colB.
+   *   df.select( df("colA") === df("colB") )
+   *   df.select( df("colA".equalTo(df("colB")) )
+   * }}}
+   */
+  override def === (other: Column): Column = EqualTo(expr, other.expr)
+
+  /**
+   * Equality test with a literal value.
+   * {{{
+   *   // The following two both select rows in which colA is "Zaharia".
+   *   df.select( df("colA") === "Zaharia")
+   *   df.select( df("colA".equalTo("Zaharia") )
+   * }}}
+   */
+  override def === (literal: Any): Column = this === Literal.anyToLiteral(literal)
+
+  /**
+   * Equality test with an expression.
+   * {{{
+   *   // The following two both select rows in which colA equals colB.
+   *   df.select( df("colA") === df("colB") )
+   *   df.select( df("colA".equalTo(df("colB")) )
+   * }}}
+   */
+  override def equalTo(other: Column): Column = this === other
+
+  /**
+   * Equality test with a literal value.
+   * {{{
+   *   // The following two both select rows in which colA is "Zaharia".
+   *   df.select( df("colA") === "Zaharia")
+   *   df.select( df("colA".equalTo("Zaharia") )
+   * }}}
+   */
+  override def equalTo(literal: Any): Column = this === literal
+
+  /**
+   * Inequality test with an expression.
+   * {{{
+   *   // The following two both select rows in which colA does not equal colB.
+   *   df.select( df("colA") !== df("colB") )
+   *   df.select( !(df("colA") === df("colB")) )
+   * }}}
+   */
+  override def !== (other: Column): Column = Not(EqualTo(expr, other.expr))
+
+  /**
+   * Inequality test with a literal value.
+   * {{{
+   *   // The following two both select rows in which colA does not equal equal 15.
+   *   df.select( df("colA") !== 15 )
+   *   df.select( !(df("colA") === 15) )
+   * }}}
+   */
+  override def !== (literal: Any): Column = this !== Literal.anyToLiteral(literal)
+
+  /**
+   * Greater than an expression.
+   * {{{
+   *   // The following selects people older than 21.
+   *   people.select( people("age") > Literal(21) )
+   * }}}
+   */
+  override def > (other: Column): Column = GreaterThan(expr, other.expr)
+
+  /**
+   * Greater than a literal value.
+   * {{{
+   *   // The following selects people older than 21.
+   *   people.select( people("age") > 21 )
+   * }}}
+   */
+  override def > (literal: Any): Column = this > Literal.anyToLiteral(literal)
+
+  /**
+   * Less than an expression.
+   * {{{
+   *   // The following selects people younger than 21.
+   *   people.select( people("age") < Literal(21) )
+   * }}}
+   */
+  override def < (other: Column): Column = LessThan(expr, other.expr)
+
+  /**
+   * Less than a literal value.
+   * {{{
+   *   // The following selects people younger than 21.
+   *   people.select( people("age") < 21 )
+   * }}}
+   */
+  override def < (literal: Any): Column = this < Literal.anyToLiteral(literal)
+
+  /**
+   * Less than or equal to an expression.
+   * {{{
+   *   // The following selects people age 21 or younger than 21.
+   *   people.select( people("age") <= Literal(21) )
+   * }}}
+   */
+  override def <= (other: Column): Column = LessThanOrEqual(expr, other.expr)
+
+  /**
+   * Less than or equal to a literal value.
+   * {{{
+   *   // The following selects people age 21 or younger than 21.
+   *   people.select( people("age") <= 21 )
+   * }}}
+   */
+  override def <= (literal: Any): Column = this <= Literal.anyToLiteral(literal)
+
+  /**
+   * Greater than or equal to an expression.
+   * {{{
+   *   // The following selects people age 21 or older than 21.
+   *   people.select( people("age") >= Literal(21) )
+   * }}}
+   */
+  override def >= (other: Column): Column = GreaterThanOrEqual(expr, other.expr)
+
+  /**
+   * Greater than or equal to a literal value.
+   * {{{
+   *   // The following selects people age 21 or older than 21.
+   *   people.select( people("age") >= 21 )
+   * }}}
+   */
+  override def >= (literal: Any): Column = this >= Literal.anyToLiteral(literal)
+
+  /**
+   * Equality test with an expression that is safe for null values.
+   */
+  override def <=> (other: Column): Column = EqualNullSafe(expr, other.expr)
+
+  /**
+   * Equality test with a literal value that is safe for null values.
+   */
+  override def <=> (literal: Any): Column = this <=> Literal.anyToLiteral(literal)
+
+  /**
+   * True if the current expression is null.
+   */
+  override def isNull: Column = IsNull(expr)
+
+  /**
+   * True if the current expression is NOT null.
+   */
+  override def isNotNull: Column = IsNotNull(expr)
+
+  /**
+   * Boolean OR with an expression.
+   * {{{
+   *   // The following selects people that are in school or employed.
+   *   people.select( people("inSchool") || people("isEmployed") )
+   * }}}
+   */
+  override def || (other: Column): Column = Or(expr, other.expr)
+
+  /**
+   * Boolean OR with a literal value.
+   * {{{
+   *   // The following selects everything.
+   *   people.select( people("inSchool") || true )
+   * }}}
+   */
+  override def || (literal: Boolean): Column = this || Literal.anyToLiteral(literal)
+
+  /**
+   * Boolean AND with an expression.
+   * {{{
+   *   // The following selects people that are in school and employed at the same time.
+   *   people.select( people("inSchool") && people("isEmployed") )
+   * }}}
+   */
+  override def && (other: Column): Column = And(expr, other.expr)
+
+  /**
+   * Boolean AND with a literal value.
+   * {{{
+   *   // The following selects people that are in school.
+   *   people.select( people("inSchool") && true )
+   * }}}
+   */
+  override def && (literal: Boolean): Column = this && Literal.anyToLiteral(literal)
+
+  /**
+   * Bitwise AND with an expression.
+   */
+  override def & (other: Column): Column = BitwiseAnd(expr, other.expr)
+
+  /**
+   * Bitwise AND with a literal value.
+   */
+  override def & (literal: Any): Column = this & Literal.anyToLiteral(literal)
+
+  /**
+   * Bitwise OR with an expression.
+   */
+  override def | (other: Column): Column = BitwiseOr(expr, other.expr)
+
+  /**
+   * Bitwise OR with a literal value.
+   */
+  override def | (literal: Any): Column = this | Literal.anyToLiteral(literal)
+
+  /**
+   * Bitwise XOR with an expression.
+   */
+  override def ^ (other: Column): Column = BitwiseXor(expr, other.expr)
+
+  /**
+   * Bitwise XOR with a literal value.
+   */
+  override def ^ (literal: Any): Column = this ^ Literal.anyToLiteral(literal)
+
+  /**
+   * Sum of this expression and another expression.
+   * {{{
+   *   // The following selects the sum of a person's height and weight.
+   *   people.select( people("height") + people("weight") )
+   * }}}
+   */
+  override def + (other: Column): Column = Add(expr, other.expr)
+
+  /**
+   * Sum of this expression and another expression.
+   * {{{
+   *   // The following selects the sum of a person's height and 10.
+   *   people.select( people("height") + 10 )
+   * }}}
+   */
+  override def + (literal: Any): Column = this + Literal.anyToLiteral(literal)
+
+  /**
+   * Subtraction. Substract the other expression from this expression.
+   * {{{
+   *   // The following selects the difference between people's height and their weight.
+   *   people.select( people("height") - people("weight") )
+   * }}}
+   */
+  override def - (other: Column): Column = Subtract(expr, other.expr)
+
+  /**
+   * Subtraction. Substract a literal value from this expression.
+   * {{{
+   *   // The following selects a person's height and substract it by 10.
+   *   people.select( people("height") - 10 )
+   * }}}
+   */
+  override def - (literal: Any): Column = this - Literal.anyToLiteral(literal)
+
+  /**
+   * Multiply this expression and another expression.
+   * {{{
+   *   // The following multiplies a person's height by their weight.
+   *   people.select( people("height") * people("weight") )
+   * }}}
+   */
+  override def * (other: Column): Column = Multiply(expr, other.expr)
+
+  /**
+   * Multiply this expression and a literal value.
+   * {{{
+   *   // The following multiplies a person's height by 10.
+   *   people.select( people("height") * 10 )
+   * }}}
+   */
+  override def * (literal: Any): Column = this * Literal.anyToLiteral(literal)
+
+  /**
+   * Divide this expression by another expression.
+   * {{{
+   *   // The following divides a person's height by their weight.
+   *   people.select( people("height") / people("weight") )
+   * }}}
+   */
+  override def / (other: Column): Column = Divide(expr, other.expr)
+
+  /**
+   * Divide this expression by a literal value.
+   * {{{
+   *   // The following divides a person's height by 10.
+   *   people.select( people("height") / 10 )
+   * }}}
+   */
+  override def / (literal: Any): Column = this / Literal.anyToLiteral(literal)
+
+  /**
+   * Modulo (a.k.a. remainder) expression.
+   */
+  override def % (other: Column): Column = Remainder(expr, other.expr)
+
+  /**
+   * Modulo (a.k.a. remainder) expression.
+   */
+  override def % (literal: Any): Column = this % Literal.anyToLiteral(literal)
+
+
+  /**
+   * A boolean expression that is evaluated to true if the value of this expression is contained
+   * by the evaluated values of the arguments.
+   */
+  @scala.annotation.varargs
+  override def in(list: Column*): Column = In(expr, list.map(_.expr))
+
+  override def like(other: Column): Column = Like(expr, other.expr)
+
+  override def like(literal: String): Column = this.like(Literal.anyToLiteral(literal))
+
+  override def rlike(other: Column): Column = RLike(expr, other.expr)
+
+  override def rlike(literal: String): Column = this.rlike(Literal.anyToLiteral(literal))
+
+
+  override def getItem(ordinal: Int): Column = GetItem(expr, LiteralExpr(ordinal))
+
+  override def getItem(ordinal: Column): Column = GetItem(expr, ordinal.expr)
+
+  override def getField(fieldName: String): Column = GetField(expr, fieldName)
+
+
+  override def substr(startPos: Column, len: Column): Column =
+    Substring(expr, startPos.expr, len.expr)
+
+  override def substr(startPos: Int, len: Int): Column =
+    this.substr(Literal.anyToLiteral(startPos), Literal.anyToLiteral(len))
+
+  override def contains(other: Column): Column = Contains(expr, other.expr)
+
+  override def contains(literal: Any): Column = this.contains(Literal.anyToLiteral(literal))
+
+
+  override def startsWith(other: Column): Column = StartsWith(expr, other.expr)
+
+  override def startsWith(literal: String): Column = this.startsWith(Literal.anyToLiteral(literal))
+
+  override def endsWith(other: Column): Column = EndsWith(expr, other.expr)
+
+  override def endsWith(literal: String): Column = this.endsWith(Literal.anyToLiteral(literal))
+
+  override def as(alias: String): Column = Alias(expr, alias)()
+
+  override def cast(to: DataType): Column = Cast(expr, to)
+
+  override def desc: Column = SortOrder(expr, Descending)
+
+  override def asc: Column = SortOrder(expr, Ascending)
+}
+
+
+class ColumnName(name: String) extends Column(name) {
+
+  /** Creates a new AttributeReference of type boolean */
+  def boolean: StructField = StructField(name, BooleanType)
+
+  /** Creates a new AttributeReference of type byte */
+  def byte: StructField = StructField(name, ByteType)
+
+  /** Creates a new AttributeReference of type short */
+  def short: StructField = StructField(name, ShortType)
+
+  /** Creates a new AttributeReference of type int */
+  def int: StructField = StructField(name, IntegerType)
+
+  /** Creates a new AttributeReference of type long */
+  def long: StructField = StructField(name, LongType)
+
+  /** Creates a new AttributeReference of type float */
+  def float: StructField = StructField(name, FloatType)
+
+  /** Creates a new AttributeReference of type double */
+  def double: StructField = StructField(name, DoubleType)
+
+  /** Creates a new AttributeReference of type string */
+  def string: StructField = StructField(name, StringType)
+
+  /** Creates a new AttributeReference of type date */
+  def date: StructField = StructField(name, DateType)
+
+  /** Creates a new AttributeReference of type decimal */
+  def decimal: StructField = StructField(name, DecimalType.Unlimited)
+
+  /** Creates a new AttributeReference of type decimal */
+  def decimal(precision: Int, scale: Int): StructField =
+    StructField(name, DecimalType(precision, scale))
+
+  /** Creates a new AttributeReference of type timestamp */
+  def timestamp: StructField = StructField(name, TimestampType)
+
+  /** Creates a new AttributeReference of type binary */
+  def binary: StructField = StructField(name, BinaryType)
+
+  /** Creates a new AttributeReference of type array */
+  def array(dataType: DataType): StructField = StructField(name, ArrayType(dataType))
+
+  /** Creates a new AttributeReference of type map */
+  def map(keyType: DataType, valueType: DataType): StructField =
+    map(MapType(keyType, valueType))
+
+  def map(mapType: MapType): StructField = StructField(name, mapType)
+
+  /** Creates a new AttributeReference of type struct */
+  def struct(fields: StructField*): StructField = struct(StructType(fields))
+
+  def struct(structType: StructType): StructField = StructField(name, structType)
+}


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