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/14 02:16:49 UTC

[1/5] spark git commit: [SPARK-5123][SQL] Reconcile Java/Scala API for data types.

Repository: spark
Updated Branches:
  refs/heads/master 14e3f114e -> f9969098c


http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 ab88f3a..efe622f 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
@@ -19,6 +19,7 @@ package org.apache.spark.sql
 
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.types._
 
 /* Implicits */
 import org.apache.spark.sql.catalyst.dsl._

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala
index 811319e..f5b945f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql
 import org.scalatest.FunSuite
 
 import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, SpecificMutableRow}
+import org.apache.spark.sql.types._
 
 class RowSuite extends FunSuite {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 bc72daf..cbdb3e6 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
@@ -23,6 +23,7 @@ import org.scalatest.BeforeAndAfterAll
 
 import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.types._
 
 /* Implicits */
 import org.apache.spark.sql.TestData._
@@ -748,7 +749,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
     val metadata = new MetadataBuilder()
       .putString(docKey, docValue)
       .build()
-    val schemaWithMeta = new StructType(Seq(
+    val schemaWithMeta = new StructType(Array(
       schema("id"), schema("name").copy(metadata = metadata), schema("age")))
     val personWithMeta = applySchema(person, schemaWithMeta)
     def validateMetadata(rdd: SchemaRDD): Unit = {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
index cf3a59e..40fb8d5 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql
 
 import java.sql.{Date, Timestamp}
 
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
 import org.scalatest.FunSuite
 
 import org.apache.spark.sql.catalyst.expressions._

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 1806a1d..a0d54d1 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,8 @@ package org.apache.spark.sql
 import scala.beans.{BeanInfo, BeanProperty}
 
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType
-import org.apache.spark.sql.catalyst.types.UserDefinedType
 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 {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala
index 8afc3a9..fdbb428 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala
@@ -17,8 +17,6 @@
 
 package org.apache.spark.sql.api.java
 
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
-
 import scala.beans.BeanProperty
 
 import org.scalatest.FunSuite
@@ -26,6 +24,7 @@ import org.scalatest.FunSuite
 import org.apache.spark.api.java.JavaSparkContext
 import org.apache.spark.sql.catalyst.util._
 import org.apache.spark.sql.test.TestSQLContext
+import org.apache.spark.sql.types.NullType
 
 // Implicits
 import scala.collection.JavaConversions._
@@ -78,10 +77,10 @@ class JavaSQLSuite extends FunSuite {
 
     schemaRDD.registerTempTable("people")
     val nullRDD = javaSqlCtx.sql("SELECT null FROM people")
-    val structFields = nullRDD.schema.getFields()
+    val structFields = nullRDD.schema.fields
     assert(structFields.size == 1)
-    assert(structFields(0).getDataType().isInstanceOf[NullType])
-    assert(nullRDD.collect.head.row === Seq(null))
+    assert(structFields(0).dataType === NullType)
+    assert(nullRDD.collect().head.row === Seq(null))
   }
 
   test("all types in JavaBeans") {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala
deleted file mode 100644
index 62fe59d..0000000
--- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala
+++ /dev/null
@@ -1,89 +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.api.java
-
-import org.scalatest.FunSuite
-
-import org.apache.spark.sql.{DataType => SDataType, StructField => SStructField, StructType => SStructType}
-import org.apache.spark.sql.types.util.DataTypeConversions._
-
-class ScalaSideDataTypeConversionSuite extends FunSuite {
-
-  def checkDataType(scalaDataType: SDataType) {
-    val javaDataType = asJavaDataType(scalaDataType)
-    val actual = asScalaDataType(javaDataType)
-    assert(scalaDataType === actual, s"Converted data type ${actual} " +
-      s"does not equal the expected data type ${scalaDataType}")
-  }
-
-  test("convert data types") {
-    // Simple DataTypes.
-    checkDataType(org.apache.spark.sql.StringType)
-    checkDataType(org.apache.spark.sql.BinaryType)
-    checkDataType(org.apache.spark.sql.BooleanType)
-    checkDataType(org.apache.spark.sql.DateType)
-    checkDataType(org.apache.spark.sql.TimestampType)
-    checkDataType(org.apache.spark.sql.DecimalType.Unlimited)
-    checkDataType(org.apache.spark.sql.DoubleType)
-    checkDataType(org.apache.spark.sql.FloatType)
-    checkDataType(org.apache.spark.sql.ByteType)
-    checkDataType(org.apache.spark.sql.IntegerType)
-    checkDataType(org.apache.spark.sql.LongType)
-    checkDataType(org.apache.spark.sql.ShortType)
-
-    // Simple ArrayType.
-    val simpleScalaArrayType =
-      org.apache.spark.sql.ArrayType(org.apache.spark.sql.StringType, true)
-    checkDataType(simpleScalaArrayType)
-
-    // Simple MapType.
-    val simpleScalaMapType =
-      org.apache.spark.sql.MapType(org.apache.spark.sql.StringType, org.apache.spark.sql.LongType)
-    checkDataType(simpleScalaMapType)
-
-    // Simple StructType.
-    val simpleScalaStructType = SStructType(
-      SStructField("a", org.apache.spark.sql.DecimalType.Unlimited, false) ::
-      SStructField("b", org.apache.spark.sql.BooleanType, true) ::
-      SStructField("c", org.apache.spark.sql.LongType, true) ::
-      SStructField("d", org.apache.spark.sql.BinaryType, false) :: Nil)
-    checkDataType(simpleScalaStructType)
-
-    // Complex StructType.
-    val metadata = new MetadataBuilder()
-      .putString("name", "age")
-      .build()
-    val complexScalaStructType = SStructType(
-      SStructField("simpleArray", simpleScalaArrayType, true) ::
-      SStructField("simpleMap", simpleScalaMapType, true) ::
-      SStructField("simpleStruct", simpleScalaStructType, true) ::
-      SStructField("boolean", org.apache.spark.sql.BooleanType, false) ::
-      SStructField("withMeta", org.apache.spark.sql.DoubleType, false, metadata) :: Nil)
-    checkDataType(complexScalaStructType)
-
-    // Complex ArrayType.
-    val complexScalaArrayType =
-      org.apache.spark.sql.ArrayType(complexScalaStructType, true)
-    checkDataType(complexScalaArrayType)
-
-    // Complex MapType.
-    val complexScalaMapType =
-      org.apache.spark.sql.MapType(complexScalaStructType, complexScalaArrayType, false)
-    checkDataType(complexScalaMapType)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala
index a9f0851..9be0b38 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.columnar
 import org.scalatest.FunSuite
 
 import org.apache.spark.sql.catalyst.expressions.Row
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 class ColumnStatsSuite extends FunSuite {
   testColumnStats(classOf[ByteColumnStats], BYTE, Row(Byte.MaxValue, Byte.MinValue, 0))

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
index 3f3f35d..87e608a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
@@ -24,9 +24,9 @@ import org.scalatest.FunSuite
 
 import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
-import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.columnar.ColumnarTestUtils._
 import org.apache.spark.sql.execution.SparkSqlSerializer
+import org.apache.spark.sql.types._
 
 class ColumnTypeSuite extends FunSuite with Logging {
   val DEFAULT_BUFFER_SIZE = 512

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
index a1f2121..f941465 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
@@ -24,7 +24,7 @@ import java.sql.{Date, Timestamp}
 
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
-import org.apache.spark.sql.catalyst.types.{DataType, NativeType}
+import org.apache.spark.sql.types.{DataType, NativeType}
 
 object ColumnarTestUtils {
   def makeNullRow(length: Int) = {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
index 21906e3..f95c895 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
@@ -22,7 +22,7 @@ import java.nio.ByteBuffer
 import org.scalatest.FunSuite
 
 import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
-import org.apache.spark.sql.catalyst.types.DataType
+import org.apache.spark.sql.types.DataType
 
 class TestNullableColumnAccessor[T <: DataType, JvmType](
     buffer: ByteBuffer,

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
index cb73f3d..80bd5c9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
@@ -19,8 +19,8 @@ package org.apache.spark.sql.columnar
 
 import org.scalatest.FunSuite
 
-import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.execution.SparkSqlSerializer
+import org.apache.spark.sql.types._
 
 class TestNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType])
   extends BasicColumnBuilder[T, JvmType](new NoopColumnStats, columnType)

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala
index 1cdb909..c82d979 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala
@@ -22,9 +22,9 @@ import java.nio.ByteBuffer
 import org.scalatest.FunSuite
 
 import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
-import org.apache.spark.sql.catalyst.types.NativeType
 import org.apache.spark.sql.columnar._
 import org.apache.spark.sql.columnar.ColumnarTestUtils._
+import org.apache.spark.sql.types.NativeType
 
 class DictionaryEncodingSuite extends FunSuite {
   testDictionaryEncoding(new IntColumnStats,    INT)

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala
index 73f31c0..8801163 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala
@@ -20,9 +20,9 @@ package org.apache.spark.sql.columnar.compression
 import org.scalatest.FunSuite
 
 import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
-import org.apache.spark.sql.catalyst.types.IntegralType
 import org.apache.spark.sql.columnar._
 import org.apache.spark.sql.columnar.ColumnarTestUtils._
+import org.apache.spark.sql.types.IntegralType
 
 class IntegralDeltaSuite extends FunSuite {
   testIntegralDelta(new IntColumnStats,  INT,  IntDelta)

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala
index 4ce2552..08df1db 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala
@@ -20,9 +20,9 @@ package org.apache.spark.sql.columnar.compression
 import org.scalatest.FunSuite
 
 import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
-import org.apache.spark.sql.catalyst.types.NativeType
 import org.apache.spark.sql.columnar._
 import org.apache.spark.sql.columnar.ColumnarTestUtils._
+import org.apache.spark.sql.types.NativeType
 
 class RunLengthEncodingSuite extends FunSuite {
   testRunLengthEncoding(new NoopColumnStats, BOOLEAN)

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala
index 7db723d..0b18b41 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala
@@ -17,8 +17,8 @@
 
 package org.apache.spark.sql.columnar.compression
 
-import org.apache.spark.sql.catalyst.types.NativeType
 import org.apache.spark.sql.columnar._
+import org.apache.spark.sql.types.NativeType
 
 class TestCompressibleColumnBuilder[T <: NativeType](
     override val columnStats: ColumnStats,

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 b09f1ac..01c1ce2 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
@@ -17,19 +17,19 @@
 
 package org.apache.spark.sql.json
 
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
-import org.apache.spark.sql.catalyst.util._
-import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, compatibleType}
-import org.apache.spark.sql.{Row, SQLConf, QueryTest}
+import java.sql.{Date, Timestamp}
+
 import org.apache.spark.sql.TestData._
+import org.apache.spark.sql.catalyst.util._
+import org.apache.spark.sql.json.JsonRDD.{compatibleType, enforceCorrectType}
 import org.apache.spark.sql.test.TestSQLContext
 import org.apache.spark.sql.test.TestSQLContext._
-
-import java.sql.{Date, Timestamp}
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
+import org.apache.spark.sql.{QueryTest, Row, SQLConf}
 
 class JsonSuite extends QueryTest {
-  import TestJsonData._
+  import org.apache.spark.sql.json.TestJsonData._
   TestJsonData
 
   test("Type promotion") {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 6ac67fc..973819a 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
@@ -21,8 +21,6 @@ import scala.collection.JavaConversions._
 import scala.reflect.ClassTag
 import scala.reflect.runtime.universe.TypeTag
 
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileSystem, Path}
 import parquet.example.data.simple.SimpleGroup
 import parquet.example.data.{Group, GroupWriter}
 import parquet.hadoop.api.WriteSupport
@@ -32,11 +30,13 @@ import parquet.hadoop.{ParquetFileWriter, ParquetWriter}
 import parquet.io.api.RecordConsumer
 import parquet.schema.{MessageType, MessageTypeParser}
 
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
 import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.expressions.Row
-import org.apache.spark.sql.catalyst.types.DecimalType
 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

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
index 0a92336..fe781ec 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
@@ -29,10 +29,10 @@ import parquet.io.api.Binary
 
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.types.IntegerType
 import org.apache.spark.sql.catalyst.util.getTempFilePath
 import org.apache.spark.sql.test.TestSQLContext
 import org.apache.spark.sql.test.TestSQLContext._
+import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
 
 case class TestRDDEntry(key: Int, value: String)
@@ -911,20 +911,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA
     Utils.deleteRecursively(tmpdir)
   }
 
-  test("DataType string parser compatibility") {
-    val schema = StructType(List(
-      StructField("c1", IntegerType, false),
-      StructField("c2", BinaryType, false)))
-
-    val fromCaseClassString = ParquetTypesConverter.convertFromString(schema.toString)
-    val fromJson = ParquetTypesConverter.convertFromString(schema.json)
-
-    (fromCaseClassString, fromJson).zipped.foreach { (a, b) =>
-      assert(a.name == b.name)
-      assert(a.dataType === b.dataType)
-    }
-  }
-
   test("read/write fixed-length decimals") {
     for ((precision, scale) <- Seq((5, 2), (1, 0), (1, 1), (18, 10), (18, 17))) {
       val tempDir = getTempFilePath("parquetTest").getCanonicalPath

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala
index 34d61bf..6427495 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala
@@ -24,7 +24,6 @@ import org.scalatest.FunSuite
 import parquet.schema.MessageTypeParser
 
 import org.apache.spark.sql.catalyst.ScalaReflection
-import org.apache.spark.sql.catalyst.types.{BinaryType, IntegerType, StructField, StructType}
 import org.apache.spark.sql.test.TestSQLContext
 
 class ParquetSchemaSuite extends FunSuite with ParquetTest {
@@ -148,12 +147,20 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest {
     """.stripMargin)
 
   test("DataType string parser compatibility") {
-    val schema = StructType(List(
-      StructField("c1", IntegerType, false),
-      StructField("c2", BinaryType, true)))
-
-    val fromCaseClassString = ParquetTypesConverter.convertFromString(schema.toString)
-    val fromJson = ParquetTypesConverter.convertFromString(schema.json)
+    // This is the generated string from previous versions of the Spark SQL, using the following:
+    // val schema = StructType(List(
+    //  StructField("c1", IntegerType, false),
+    //  StructField("c2", BinaryType, true)))
+    val caseClassString =
+      "StructType(List(StructField(c1,IntegerType,false), StructField(c2,BinaryType,true)))"
+
+    val jsonString =
+      """
+        |{"type":"struct","fields":[{"name":"c1","type":"integer","nullable":false,"metadata":{}},{"name":"c2","type":"binary","nullable":true,"metadata":{}}]}
+      """.stripMargin
+
+    val fromCaseClassString = ParquetTypesConverter.convertFromString(caseClassString)
+    val fromJson = ParquetTypesConverter.convertFromString(jsonString)
 
     (fromCaseClassString, fromJson).zipped.foreach { (a, b) =>
       assert(a.name == b.name)

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala
index 939b3c0..390538d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala
@@ -20,6 +20,8 @@ package org.apache.spark.sql.sources
 import scala.language.existentials
 
 import org.apache.spark.sql._
+import org.apache.spark.sql.types._
+
 
 class FilteredScanSource extends RelationProvider {
   override def createRelation(

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 fee2e22..7900b3e 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
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.sources
 
 import org.apache.spark.sql._
+import org.apache.spark.sql.types._
 
 class PrunedScanSource extends RelationProvider {
   override def createRelation(

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
index a1d2468..382dddc 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.sources
 import java.sql.{Timestamp, Date}
 
 import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.types.DecimalType
+import org.apache.spark.sql.types._
 
 class DefaultSource extends SimpleScanSource
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 742acba..171d707 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,11 +32,11 @@ 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.catalyst.types._
+import org.apache.spark.sql.{SQLConf, SchemaRDD, 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}
-import org.apache.spark.sql.{SQLConf, SchemaRDD, Row => SparkRow}
+import org.apache.spark.sql.types._
 
 /**
  * A compatibility layer for interacting with Hive version 0.12.0.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 b821564..bec9d9a 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,11 +30,11 @@ 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.execution.SetCommand
-import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
 import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes}
-import org.apache.spark.sql.{Row => SparkRow, SQLConf, SchemaRDD}
+import org.apache.spark.sql.types._
 
 /**
  * A compatibility layer for interacting with Hive version 0.13.1.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 9aeebd7..bf56e60 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
@@ -37,10 +37,10 @@ import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators, OverrideCatalog, OverrideFunctionRegistry}
 import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.catalyst.types.DecimalType
 import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUdfs, SetCommand, QueryExecutionException}
 import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DescribeHiveTableCommand}
 import org.apache.spark.sql.sources.DataSourceStrategy
+import org.apache.spark.sql.types._
 
 /**
  * DEPRECATED: Use HiveContext instead.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index a156d6f..245b847 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
@@ -24,9 +24,9 @@ import org.apache.hadoop.hive.serde2.{io => hiveIo}
 import org.apache.hadoop.{io => hadoopIo}
 
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.types
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.apache.spark.sql.types
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
 
 /* Implicit conversions */
 import scala.collection.JavaConversions._
@@ -43,7 +43,7 @@ import scala.collection.JavaConversions._
  *     long / scala.Long
  *     short / scala.Short
  *     byte / scala.Byte
- *     org.apache.spark.sql.catalyst.types.decimal.Decimal
+ *     org.apache.spark.sql.types.decimal.Decimal
  *     Array[Byte]
  *     java.sql.Date
  *     java.sql.Timestamp
@@ -504,7 +504,8 @@ private[hive] trait HiveInspectors {
     case DecimalType() => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector
     case StructType(fields) =>
       ObjectInspectorFactory.getStandardStructObjectInspector(
-        fields.map(f => f.name), fields.map(f => toInspector(f.dataType)))
+        java.util.Arrays.asList(fields.map(f => f.name) :_*),
+        java.util.Arrays.asList(fields.map(f => toInspector(f.dataType)) :_*))
   }
 
   /**
@@ -618,7 +619,9 @@ private[hive] trait HiveInspectors {
       case ArrayType(elemType, _) =>
         getListTypeInfo(elemType.toTypeInfo)
       case StructType(fields) =>
-        getStructTypeInfo(fields.map(_.name), fields.map(_.dataType.toTypeInfo))
+        getStructTypeInfo(
+          java.util.Arrays.asList(fields.map(_.name) :_*),
+          java.util.Arrays.asList(fields.map(_.dataType.toTypeInfo) :_*))
       case MapType(keyType, valueType, _) =>
         getMapTypeInfo(keyType.toTypeInfo, valueType.toTypeInfo)
       case BinaryType => binaryTypeInfo

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 785a6a1..d40f993 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -39,8 +39,8 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.sources.{LogicalRelation, ResolvedDataSource}
+import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
 
 /* Implicit conversions */

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index 34622b5..b13ef72 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.hive
 
 import java.sql.Date
+
 import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hadoop.hive.ql.Context
 import org.apache.hadoop.hive.ql.lib.Node
@@ -31,10 +32,10 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical
 import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
 import org.apache.spark.sql.execution.ExplainCommand
 import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable}
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
 
 /* Implicit conversions */
 import scala.collection.JavaConversions._

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 cdff82e..6952b12 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
@@ -17,23 +17,24 @@
 
 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.catalyst.analysis.UnresolvedAttribute
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
 import org.apache.spark.sql.catalyst.planning._
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.types.StringType
 import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand}
 import org.apache.spark.sql.execution._
 import org.apache.spark.sql.hive
 import org.apache.spark.sql.hive.execution._
 import org.apache.spark.sql.parquet.ParquetRelation
 import org.apache.spark.sql.sources.CreateTableUsing
-import org.apache.spark.sql.{SQLContext, SchemaRDD, Strategy}
+import org.apache.spark.sql.types.StringType
 
-import scala.collection.JavaConversions._
 
 private[hive] trait HiveStrategies {
   // Possibly being too clever with types here... or not clever enough.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
index 8ba818a..781a2e9 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row}
 import org.apache.spark.sql.execution.RunnableCommand
 import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.sql.SQLContext
-import org.apache.spark.sql.catalyst.types.StringType
+import org.apache.spark.sql.types.StringType
 
 /**
  * :: DeveloperApi ::

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
index 8bbcd6f..b56175f 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
@@ -19,20 +19,18 @@ package org.apache.spark.sql.hive.execution
 
 import scala.collection.JavaConversions._
 
-import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar}
 import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition}
 import org.apache.hadoop.hive.serde.serdeConstants
 import org.apache.hadoop.hive.serde2.objectinspector._
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption
-import org.apache.hadoop.hive.serde2.objectinspector.primitive._
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils
 
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.types.{BooleanType, DataType}
 import org.apache.spark.sql.execution._
 import org.apache.spark.sql.hive._
+import org.apache.spark.sql.types.{BooleanType, DataType}
 
 /**
  * :: DeveloperApi ::

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
index e70cdea..cf72345 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
@@ -18,11 +18,11 @@
 package org.apache.spark.sql.hive.execution
 
 import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.sql.SQLContext
 import org.apache.spark.sql.catalyst.expressions.Row
-import org.apache.spark.sql.catalyst.types.StructType
 import org.apache.spark.sql.execution.RunnableCommand
 import org.apache.spark.sql.hive.HiveContext
-import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.types.StructType
 
 /**
  * :: DeveloperApi ::

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
index 7d863f9..d898b87 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
@@ -33,7 +33,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._
 import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.analysis
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils.getContextOrSparkClassLoader
 
 /* Implicit conversions */

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
index f90d360..dc23d9a 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
@@ -17,22 +17,21 @@
 
 package org.apache.spark.sql.hive
 
-import java.sql.Date
 import java.util
+import java.sql.Date
 import java.util.{Locale, TimeZone}
 
-import org.apache.hadoop.hive.serde2.io.DoubleWritable
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
-import org.scalatest.FunSuite
-
 import org.apache.hadoop.hive.ql.udf.UDAFPercentile
-import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector, ObjectInspectorFactory}
+import org.apache.hadoop.hive.serde2.io.DoubleWritable
+import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory, StructObjectInspector}
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory
 import org.apache.hadoop.io.LongWritable
+import org.scalatest.FunSuite
 
 import org.apache.spark.sql.catalyst.expressions.{Literal, Row}
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
 
 class HiveInspectorSuite extends FunSuite with HiveInspectors {
   test("Test wrap SettableStructObjectInspector") {
@@ -93,7 +92,6 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors {
   val row = data.map(_.eval(null))
   val dataTypes = data.map(_.dataType)
 
-  import scala.collection.JavaConversions._
   def toWritableInspector(dataType: DataType): ObjectInspector = dataType match {
     case ArrayType(tpe, _) =>
       ObjectInspectorFactory.getStandardListObjectInspector(toWritableInspector(tpe))
@@ -115,7 +113,8 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors {
     case DecimalType() => PrimitiveObjectInspectorFactory.writableHiveDecimalObjectInspector
     case StructType(fields) =>
       ObjectInspectorFactory.getStandardStructObjectInspector(
-        fields.map(f => f.name), fields.map(f => toWritableInspector(f.dataType)))
+        java.util.Arrays.asList(fields.map(f => f.name) :_*),
+        java.util.Arrays.asList(fields.map(f => toWritableInspector(f.dataType)) :_*))
   }
 
   def checkDataType(dt1: Seq[DataType], dt2: Seq[DataType]): Unit = {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
index 041a36f..fa6905f 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
@@ -19,9 +19,9 @@ package org.apache.spark.sql.hive
 
 import org.scalatest.FunSuite
 
-import org.apache.spark.sql.catalyst.types.StructType
 import org.apache.spark.sql.sources.DDLParser
 import org.apache.spark.sql.test.ExamplePointUDT
+import org.apache.spark.sql.types.StructType
 
 class HiveMetastoreCatalogSuite extends FunSuite {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 fb481ed..7cfb875 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
@@ -22,6 +22,7 @@ import java.io.File
 import com.google.common.io.Files
 import org.apache.spark.sql.{QueryTest, _}
 import org.apache.spark.sql.hive.test.TestHive
+import org.apache.spark.sql.types._
 
 /* Implicits */
 import org.apache.spark.sql.hive.test.TestHive._

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index ec9ebb4..8ff833e 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -25,6 +25,7 @@ import org.apache.commons.io.FileUtils
 
 import org.apache.spark.sql._
 import org.apache.spark.util.Utils
+import org.apache.spark.sql.types._
 
 /* Implicits */
 import org.apache.spark.sql.hive.test.TestHive._

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala
----------------------------------------------------------------------
diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala
index 25fdf5c..a558746 100644
--- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala
+++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala
@@ -18,8 +18,12 @@
 package org.apache.spark.sql.hive
 
 import java.net.URI
-import java.util.{ArrayList => JArrayList}
-import java.util.Properties
+import java.util.{ArrayList => JArrayList, Properties}
+
+import scala.collection.JavaConversions._
+import scala.language.implicitConversions
+
+import org.apache.hadoop.{io => hadoopIo}
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.hive.common.`type`.HiveDecimal
@@ -29,20 +33,16 @@ import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table}
 import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc}
 import org.apache.hadoop.hive.ql.processors._
 import org.apache.hadoop.hive.ql.stats.StatsSetupConst
+import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer, io => hiveIo}
+import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, PrimitiveObjectInspector}
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory}
-import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector}
 import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory}
-import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils}
-import org.apache.hadoop.hive.serde2.{io => hiveIo}
 import org.apache.hadoop.io.NullWritable
-import org.apache.hadoop.{io => hadoopIo}
 import org.apache.hadoop.mapred.InputFormat
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
-import scala.collection.JavaConversions._
-import scala.language.implicitConversions
 
-import org.apache.spark.sql.catalyst.types.DecimalType
+import org.apache.spark.sql.types.DecimalType
+import org.apache.spark.sql.types.decimal.Decimal
 
 case class HiveFunctionWrapper(functionClassName: String) extends java.io.Serializable {
   // for Serialization

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala
----------------------------------------------------------------------
diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala
index e47002c..a712136 100644
--- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala
+++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala
@@ -20,6 +20,9 @@ package org.apache.spark.sql.hive
 import java.util.{ArrayList => JArrayList}
 import java.util.Properties
 
+import scala.collection.JavaConversions._
+import scala.language.implicitConversions
+
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.io.NullWritable
@@ -37,12 +40,10 @@ import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector,
 import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils}
 import org.apache.hadoop.hive.serde2.{io => hiveIo}
 import org.apache.hadoop.{io => hadoopIo}
-import org.apache.spark.Logging
-import org.apache.spark.sql.catalyst.types.DecimalType
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
 
-import scala.collection.JavaConversions._
-import scala.language.implicitConversions
+import org.apache.spark.Logging
+import org.apache.spark.sql.types.DecimalType
+import org.apache.spark.sql.types.decimal.Decimal
 
 
 /**


---------------------------------------------------------------------
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-5123][SQL] Reconcile Java/Scala API for data types.

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index f430057..3aea337 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -22,7 +22,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite}
 import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference}
 import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.dsl.plans._

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala
index bbbeb4f..bc2ec75 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis
 
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical.{Project, LocalRelation}
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 import org.scalatest.{BeforeAndAfter, FunSuite}
 
 class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
index dfa2d95..f5a502b 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
@@ -21,7 +21,7 @@ import org.scalatest.FunSuite
 
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project}
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 class HiveTypeCoercionSuite extends FunSuite {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
index 4ba7d87..8552448 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
@@ -21,16 +21,14 @@ import java.sql.{Date, Timestamp}
 
 import scala.collection.immutable.HashSet
 
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.scalactic.TripleEqualsSupport.Spread
 import org.scalatest.FunSuite
 import org.scalatest.Matchers._
-import org.scalactic.TripleEqualsSupport.Spread
-
-import org.apache.spark.sql.catalyst.types._
 
-
-/* Implicit conversions */
 import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
+
 
 class ExpressionEvaluationSuite extends FunSuite {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
index 0a27cce..9fdf3ef 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
 import org.apache.spark.sql.catalyst.plans.PlanTest
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 // For implicit conversions
 import org.apache.spark.sql.catalyst.dsl.plans._

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala
index 017b180..da912ab 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
 import org.apache.spark.sql.catalyst.plans.PlanTest
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 // For implicit conversions
 import org.apache.spark.sql.catalyst.dsl.plans._

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
index 036fd3f..cdb843f 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
@@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer
 import org.scalatest.FunSuite
 
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.types.{StringType, NullType}
+import org.apache.spark.sql.types.{StringType, NullType}
 
 case class Dummy(optKey: Option[Expression]) extends Expression {
   def children = optKey.toSeq

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala
deleted file mode 100644
index e32f1ac..0000000
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala
+++ /dev/null
@@ -1,157 +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.catalyst.types.decimal
-
-import org.scalatest.{PrivateMethodTester, FunSuite}
-
-import scala.language.postfixOps
-
-class DecimalSuite extends FunSuite with PrivateMethodTester {
-  test("creating decimals") {
-    /** Check that a Decimal has the given string representation, precision and scale */
-    def checkDecimal(d: Decimal, string: String, precision: Int, scale: Int): Unit = {
-      assert(d.toString === string)
-      assert(d.precision === precision)
-      assert(d.scale === scale)
-    }
-
-    checkDecimal(new Decimal(), "0", 1, 0)
-    checkDecimal(Decimal(BigDecimal("10.030")), "10.030", 5, 3)
-    checkDecimal(Decimal(BigDecimal("10.030"), 4, 1), "10.0", 4, 1)
-    checkDecimal(Decimal(BigDecimal("-9.95"), 4, 1), "-10.0", 4, 1)
-    checkDecimal(Decimal("10.030"), "10.030", 5, 3)
-    checkDecimal(Decimal(10.03), "10.03", 4, 2)
-    checkDecimal(Decimal(17L), "17", 20, 0)
-    checkDecimal(Decimal(17), "17", 10, 0)
-    checkDecimal(Decimal(17L, 2, 1), "1.7", 2, 1)
-    checkDecimal(Decimal(170L, 4, 2), "1.70", 4, 2)
-    checkDecimal(Decimal(17L, 24, 1), "1.7", 24, 1)
-    checkDecimal(Decimal(1e17.toLong, 18, 0), 1e17.toLong.toString, 18, 0)
-    checkDecimal(Decimal(Long.MaxValue), Long.MaxValue.toString, 20, 0)
-    checkDecimal(Decimal(Long.MinValue), Long.MinValue.toString, 20, 0)
-    intercept[IllegalArgumentException](Decimal(170L, 2, 1))
-    intercept[IllegalArgumentException](Decimal(170L, 2, 0))
-    intercept[IllegalArgumentException](Decimal(BigDecimal("10.030"), 2, 1))
-    intercept[IllegalArgumentException](Decimal(BigDecimal("-9.95"), 2, 1))
-    intercept[IllegalArgumentException](Decimal(1e17.toLong, 17, 0))
-  }
-
-  test("double and long values") {
-    /** Check that a Decimal converts to the given double and long values */
-    def checkValues(d: Decimal, doubleValue: Double, longValue: Long): Unit = {
-      assert(d.toDouble === doubleValue)
-      assert(d.toLong === longValue)
-    }
-
-    checkValues(new Decimal(), 0.0, 0L)
-    checkValues(Decimal(BigDecimal("10.030")), 10.03, 10L)
-    checkValues(Decimal(BigDecimal("10.030"), 4, 1), 10.0, 10L)
-    checkValues(Decimal(BigDecimal("-9.95"), 4, 1), -10.0, -10L)
-    checkValues(Decimal(10.03), 10.03, 10L)
-    checkValues(Decimal(17L), 17.0, 17L)
-    checkValues(Decimal(17), 17.0, 17L)
-    checkValues(Decimal(17L, 2, 1), 1.7, 1L)
-    checkValues(Decimal(170L, 4, 2), 1.7, 1L)
-    checkValues(Decimal(1e16.toLong), 1e16, 1e16.toLong)
-    checkValues(Decimal(1e17.toLong), 1e17, 1e17.toLong)
-    checkValues(Decimal(1e18.toLong), 1e18, 1e18.toLong)
-    checkValues(Decimal(2e18.toLong), 2e18, 2e18.toLong)
-    checkValues(Decimal(Long.MaxValue), Long.MaxValue.toDouble, Long.MaxValue)
-    checkValues(Decimal(Long.MinValue), Long.MinValue.toDouble, Long.MinValue)
-    checkValues(Decimal(Double.MaxValue), Double.MaxValue, 0L)
-    checkValues(Decimal(Double.MinValue), Double.MinValue, 0L)
-  }
-
-  // Accessor for the BigDecimal value of a Decimal, which will be null if it's using Longs
-  private val decimalVal = PrivateMethod[BigDecimal]('decimalVal)
-
-  /** Check whether a decimal is represented compactly (passing whether we expect it to be) */
-  private def checkCompact(d: Decimal, expected: Boolean): Unit = {
-    val isCompact = d.invokePrivate(decimalVal()).eq(null)
-    assert(isCompact == expected, s"$d ${if (expected) "was not" else "was"} compact")
-  }
-
-  test("small decimals represented as unscaled long") {
-    checkCompact(new Decimal(), true)
-    checkCompact(Decimal(BigDecimal(10.03)), false)
-    checkCompact(Decimal(BigDecimal(1e20)), false)
-    checkCompact(Decimal(17L), true)
-    checkCompact(Decimal(17), true)
-    checkCompact(Decimal(17L, 2, 1), true)
-    checkCompact(Decimal(170L, 4, 2), true)
-    checkCompact(Decimal(17L, 24, 1), true)
-    checkCompact(Decimal(1e16.toLong), true)
-    checkCompact(Decimal(1e17.toLong), true)
-    checkCompact(Decimal(1e18.toLong - 1), true)
-    checkCompact(Decimal(- 1e18.toLong + 1), true)
-    checkCompact(Decimal(1e18.toLong - 1, 30, 10), true)
-    checkCompact(Decimal(- 1e18.toLong + 1, 30, 10), true)
-    checkCompact(Decimal(1e18.toLong), false)
-    checkCompact(Decimal(-1e18.toLong), false)
-    checkCompact(Decimal(1e18.toLong, 30, 10), false)
-    checkCompact(Decimal(-1e18.toLong, 30, 10), false)
-    checkCompact(Decimal(Long.MaxValue), false)
-    checkCompact(Decimal(Long.MinValue), false)
-  }
-
-  test("hash code") {
-    assert(Decimal(123).hashCode() === (123).##)
-    assert(Decimal(-123).hashCode() === (-123).##)
-    assert(Decimal(Int.MaxValue).hashCode() === Int.MaxValue.##)
-    assert(Decimal(Long.MaxValue).hashCode() === Long.MaxValue.##)
-    assert(Decimal(BigDecimal(123)).hashCode() === (123).##)
-
-    val reallyBig = BigDecimal("123182312312313232112312312123.1231231231")
-    assert(Decimal(reallyBig).hashCode() === reallyBig.hashCode)
-  }
-
-  test("equals") {
-    // The decimals on the left are stored compactly, while the ones on the right aren't
-    checkCompact(Decimal(123), true)
-    checkCompact(Decimal(BigDecimal(123)), false)
-    checkCompact(Decimal("123"), false)
-    assert(Decimal(123) === Decimal(BigDecimal(123)))
-    assert(Decimal(123) === Decimal(BigDecimal("123.00")))
-    assert(Decimal(-123) === Decimal(BigDecimal(-123)))
-    assert(Decimal(-123) === Decimal(BigDecimal("-123.00")))
-  }
-
-  test("isZero") {
-    assert(Decimal(0).isZero)
-    assert(Decimal(0, 4, 2).isZero)
-    assert(Decimal("0").isZero)
-    assert(Decimal("0.000").isZero)
-    assert(!Decimal(1).isZero)
-    assert(!Decimal(1, 4, 2).isZero)
-    assert(!Decimal("1").isZero)
-    assert(!Decimal("0.001").isZero)
-  }
-
-  test("arithmetic") {
-    assert(Decimal(100) + Decimal(-100) === Decimal(0))
-    assert(Decimal(100) + Decimal(-100) === Decimal(0))
-    assert(Decimal(100) * Decimal(-100) === Decimal(-10000))
-    assert(Decimal(1e13) * Decimal(1e13) === Decimal(1e26))
-    assert(Decimal(100) / Decimal(-100) === Decimal(-1))
-    assert(Decimal(100) / Decimal(0) === null)
-    assert(Decimal(100) % Decimal(-100) === Decimal(0))
-    assert(Decimal(100) % Decimal(3) === Decimal(1))
-    assert(Decimal(-100) % Decimal(3) === Decimal(-1))
-    assert(Decimal(100) % Decimal(0) === null)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala
index f005b7d..d7d60ef 100755
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala
@@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.util
 import org.json4s.jackson.JsonMethods.parse
 import org.scalatest.FunSuite
 
+import org.apache.spark.sql.types.{MetadataBuilder, Metadata}
+
 class MetadataSuite extends FunSuite {
 
   val baseMetadata = new MetadataBuilder()

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala
new file mode 100644
index 0000000..892195f
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala
@@ -0,0 +1,88 @@
+/*
+* 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.types
+
+import org.scalatest.FunSuite
+
+class DataTypeSuite extends FunSuite {
+
+  test("construct an ArrayType") {
+    val array = ArrayType(StringType)
+
+    assert(ArrayType(StringType, true) === array)
+  }
+
+  test("construct an MapType") {
+    val map = MapType(StringType, IntegerType)
+
+    assert(MapType(StringType, IntegerType, true) === map)
+  }
+
+  test("extract fields from a StructType") {
+    val struct = StructType(
+      StructField("a", IntegerType, true) ::
+      StructField("b", LongType, false) ::
+      StructField("c", StringType, true) ::
+      StructField("d", FloatType, true) :: Nil)
+
+    assert(StructField("b", LongType, false) === struct("b"))
+
+    intercept[IllegalArgumentException] {
+      struct("e")
+    }
+
+    val expectedStruct = StructType(
+      StructField("b", LongType, false) ::
+      StructField("d", FloatType, true) :: Nil)
+
+    assert(expectedStruct === struct(Set("b", "d")))
+    intercept[IllegalArgumentException] {
+      struct(Set("b", "d", "e", "f"))
+    }
+  }
+
+  def checkDataTypeJsonRepr(dataType: DataType): Unit = {
+    test(s"JSON - $dataType") {
+      assert(DataType.fromJson(dataType.json) === dataType)
+    }
+  }
+
+  checkDataTypeJsonRepr(BooleanType)
+  checkDataTypeJsonRepr(ByteType)
+  checkDataTypeJsonRepr(ShortType)
+  checkDataTypeJsonRepr(IntegerType)
+  checkDataTypeJsonRepr(LongType)
+  checkDataTypeJsonRepr(FloatType)
+  checkDataTypeJsonRepr(DoubleType)
+  checkDataTypeJsonRepr(DecimalType.Unlimited)
+  checkDataTypeJsonRepr(TimestampType)
+  checkDataTypeJsonRepr(StringType)
+  checkDataTypeJsonRepr(BinaryType)
+  checkDataTypeJsonRepr(ArrayType(DoubleType, true))
+  checkDataTypeJsonRepr(ArrayType(StringType, false))
+  checkDataTypeJsonRepr(MapType(IntegerType, StringType, true))
+  checkDataTypeJsonRepr(MapType(IntegerType, ArrayType(DoubleType), false))
+  val metadata = new MetadataBuilder()
+    .putString("name", "age")
+    .build()
+  checkDataTypeJsonRepr(
+    StructType(Seq(
+      StructField("a", IntegerType, nullable = true),
+      StructField("b", ArrayType(DoubleType), nullable = false),
+      StructField("c", DoubleType, nullable = false, metadata))))
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
new file mode 100644
index 0000000..813377d
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
@@ -0,0 +1,157 @@
+/*
+ * 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.types.decimal
+
+import org.scalatest.{PrivateMethodTester, FunSuite}
+
+import scala.language.postfixOps
+
+class DecimalSuite extends FunSuite with PrivateMethodTester {
+  test("creating decimals") {
+    /** Check that a Decimal has the given string representation, precision and scale */
+    def checkDecimal(d: Decimal, string: String, precision: Int, scale: Int): Unit = {
+      assert(d.toString === string)
+      assert(d.precision === precision)
+      assert(d.scale === scale)
+    }
+
+    checkDecimal(new Decimal(), "0", 1, 0)
+    checkDecimal(Decimal(BigDecimal("10.030")), "10.030", 5, 3)
+    checkDecimal(Decimal(BigDecimal("10.030"), 4, 1), "10.0", 4, 1)
+    checkDecimal(Decimal(BigDecimal("-9.95"), 4, 1), "-10.0", 4, 1)
+    checkDecimal(Decimal("10.030"), "10.030", 5, 3)
+    checkDecimal(Decimal(10.03), "10.03", 4, 2)
+    checkDecimal(Decimal(17L), "17", 20, 0)
+    checkDecimal(Decimal(17), "17", 10, 0)
+    checkDecimal(Decimal(17L, 2, 1), "1.7", 2, 1)
+    checkDecimal(Decimal(170L, 4, 2), "1.70", 4, 2)
+    checkDecimal(Decimal(17L, 24, 1), "1.7", 24, 1)
+    checkDecimal(Decimal(1e17.toLong, 18, 0), 1e17.toLong.toString, 18, 0)
+    checkDecimal(Decimal(Long.MaxValue), Long.MaxValue.toString, 20, 0)
+    checkDecimal(Decimal(Long.MinValue), Long.MinValue.toString, 20, 0)
+    intercept[IllegalArgumentException](Decimal(170L, 2, 1))
+    intercept[IllegalArgumentException](Decimal(170L, 2, 0))
+    intercept[IllegalArgumentException](Decimal(BigDecimal("10.030"), 2, 1))
+    intercept[IllegalArgumentException](Decimal(BigDecimal("-9.95"), 2, 1))
+    intercept[IllegalArgumentException](Decimal(1e17.toLong, 17, 0))
+  }
+
+  test("double and long values") {
+    /** Check that a Decimal converts to the given double and long values */
+    def checkValues(d: Decimal, doubleValue: Double, longValue: Long): Unit = {
+      assert(d.toDouble === doubleValue)
+      assert(d.toLong === longValue)
+    }
+
+    checkValues(new Decimal(), 0.0, 0L)
+    checkValues(Decimal(BigDecimal("10.030")), 10.03, 10L)
+    checkValues(Decimal(BigDecimal("10.030"), 4, 1), 10.0, 10L)
+    checkValues(Decimal(BigDecimal("-9.95"), 4, 1), -10.0, -10L)
+    checkValues(Decimal(10.03), 10.03, 10L)
+    checkValues(Decimal(17L), 17.0, 17L)
+    checkValues(Decimal(17), 17.0, 17L)
+    checkValues(Decimal(17L, 2, 1), 1.7, 1L)
+    checkValues(Decimal(170L, 4, 2), 1.7, 1L)
+    checkValues(Decimal(1e16.toLong), 1e16, 1e16.toLong)
+    checkValues(Decimal(1e17.toLong), 1e17, 1e17.toLong)
+    checkValues(Decimal(1e18.toLong), 1e18, 1e18.toLong)
+    checkValues(Decimal(2e18.toLong), 2e18, 2e18.toLong)
+    checkValues(Decimal(Long.MaxValue), Long.MaxValue.toDouble, Long.MaxValue)
+    checkValues(Decimal(Long.MinValue), Long.MinValue.toDouble, Long.MinValue)
+    checkValues(Decimal(Double.MaxValue), Double.MaxValue, 0L)
+    checkValues(Decimal(Double.MinValue), Double.MinValue, 0L)
+  }
+
+  // Accessor for the BigDecimal value of a Decimal, which will be null if it's using Longs
+  private val decimalVal = PrivateMethod[BigDecimal]('decimalVal)
+
+  /** Check whether a decimal is represented compactly (passing whether we expect it to be) */
+  private def checkCompact(d: Decimal, expected: Boolean): Unit = {
+    val isCompact = d.invokePrivate(decimalVal()).eq(null)
+    assert(isCompact == expected, s"$d ${if (expected) "was not" else "was"} compact")
+  }
+
+  test("small decimals represented as unscaled long") {
+    checkCompact(new Decimal(), true)
+    checkCompact(Decimal(BigDecimal(10.03)), false)
+    checkCompact(Decimal(BigDecimal(1e20)), false)
+    checkCompact(Decimal(17L), true)
+    checkCompact(Decimal(17), true)
+    checkCompact(Decimal(17L, 2, 1), true)
+    checkCompact(Decimal(170L, 4, 2), true)
+    checkCompact(Decimal(17L, 24, 1), true)
+    checkCompact(Decimal(1e16.toLong), true)
+    checkCompact(Decimal(1e17.toLong), true)
+    checkCompact(Decimal(1e18.toLong - 1), true)
+    checkCompact(Decimal(- 1e18.toLong + 1), true)
+    checkCompact(Decimal(1e18.toLong - 1, 30, 10), true)
+    checkCompact(Decimal(- 1e18.toLong + 1, 30, 10), true)
+    checkCompact(Decimal(1e18.toLong), false)
+    checkCompact(Decimal(-1e18.toLong), false)
+    checkCompact(Decimal(1e18.toLong, 30, 10), false)
+    checkCompact(Decimal(-1e18.toLong, 30, 10), false)
+    checkCompact(Decimal(Long.MaxValue), false)
+    checkCompact(Decimal(Long.MinValue), false)
+  }
+
+  test("hash code") {
+    assert(Decimal(123).hashCode() === (123).##)
+    assert(Decimal(-123).hashCode() === (-123).##)
+    assert(Decimal(Int.MaxValue).hashCode() === Int.MaxValue.##)
+    assert(Decimal(Long.MaxValue).hashCode() === Long.MaxValue.##)
+    assert(Decimal(BigDecimal(123)).hashCode() === (123).##)
+
+    val reallyBig = BigDecimal("123182312312313232112312312123.1231231231")
+    assert(Decimal(reallyBig).hashCode() === reallyBig.hashCode)
+  }
+
+  test("equals") {
+    // The decimals on the left are stored compactly, while the ones on the right aren't
+    checkCompact(Decimal(123), true)
+    checkCompact(Decimal(BigDecimal(123)), false)
+    checkCompact(Decimal("123"), false)
+    assert(Decimal(123) === Decimal(BigDecimal(123)))
+    assert(Decimal(123) === Decimal(BigDecimal("123.00")))
+    assert(Decimal(-123) === Decimal(BigDecimal(-123)))
+    assert(Decimal(-123) === Decimal(BigDecimal("-123.00")))
+  }
+
+  test("isZero") {
+    assert(Decimal(0).isZero)
+    assert(Decimal(0, 4, 2).isZero)
+    assert(Decimal("0").isZero)
+    assert(Decimal("0.000").isZero)
+    assert(!Decimal(1).isZero)
+    assert(!Decimal(1, 4, 2).isZero)
+    assert(!Decimal("1").isZero)
+    assert(!Decimal("0.001").isZero)
+  }
+
+  test("arithmetic") {
+    assert(Decimal(100) + Decimal(-100) === Decimal(0))
+    assert(Decimal(100) + Decimal(-100) === Decimal(0))
+    assert(Decimal(100) * Decimal(-100) === Decimal(-10000))
+    assert(Decimal(1e13) * Decimal(1e13) === Decimal(1e26))
+    assert(Decimal(100) / Decimal(-100) === Decimal(-1))
+    assert(Decimal(100) / Decimal(0) === null)
+    assert(Decimal(100) % Decimal(-100) === Decimal(0))
+    assert(Decimal(100) % Decimal(3) === Decimal(1))
+    assert(Decimal(-100) % Decimal(3) === Decimal(-1))
+    assert(Decimal(100) % Decimal(0) === null)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java
deleted file mode 100644
index b73a371..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java
+++ /dev/null
@@ -1,68 +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.api.java;
-
-/**
- * The data type representing Lists.
- * An ArrayType object comprises two fields, {@code DataType elementType} and
- * {@code boolean containsNull}. The field of {@code elementType} is used to specify the type of
- * array elements. The field of {@code containsNull} is used to specify if the array has
- * {@code null} values.
- *
- * To create an {@link ArrayType},
- * {@link DataType#createArrayType(DataType)} or
- * {@link DataType#createArrayType(DataType, boolean)}
- * should be used.
- */
-public class ArrayType extends DataType {
-  private DataType elementType;
-  private boolean containsNull;
-
-  protected ArrayType(DataType elementType, boolean containsNull) {
-    this.elementType = elementType;
-    this.containsNull = containsNull;
-  }
-
-  public DataType getElementType() {
-    return elementType;
-  }
-
-  public boolean isContainsNull() {
-    return containsNull;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    ArrayType arrayType = (ArrayType) o;
-
-    if (containsNull != arrayType.containsNull) return false;
-    if (!elementType.equals(arrayType.elementType)) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = elementType.hashCode();
-    result = 31 * result + (containsNull ? 1 : 0);
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java
deleted file mode 100644
index 7daad60..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java
+++ /dev/null
@@ -1,27 +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.api.java;
-
-/**
- * The data type representing byte[] values.
- *
- * {@code BinaryType} is represented by the singleton object {@link DataType#BinaryType}.
- */
-public class BinaryType extends DataType {
-  protected BinaryType() {}
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java
deleted file mode 100644
index 5a1f527..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java
+++ /dev/null
@@ -1,27 +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.api.java;
-
-/**
- * The data type representing boolean and Boolean values.
- *
- * {@code BooleanType} is represented by the singleton object {@link DataType#BooleanType}.
- */
-public class BooleanType extends DataType {
-  protected BooleanType() {}
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java
deleted file mode 100644
index e5cdf06..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java
+++ /dev/null
@@ -1,27 +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.api.java;
-
-/**
- * The data type representing byte and Byte values.
- *
- * {@code ByteType} is represented by the singleton object {@link DataType#ByteType}.
- */
-public class ByteType extends DataType {
-  protected ByteType() {}
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java
deleted file mode 100644
index c69bbd5..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java
+++ /dev/null
@@ -1,208 +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.api.java;
-
-import java.util.*;
-
-/**
- * The base type of all Spark SQL data types.
- *
- * To get/create specific data type, users should use singleton objects and factory methods
- * provided by this class.
- */
-public abstract class DataType {
-
-  /**
-   * Gets the StringType object.
-   */
-  public static final StringType StringType = new StringType();
-
-  /**
-   * Gets the BinaryType object.
-   */
-  public static final BinaryType BinaryType = new BinaryType();
-
-  /**
-   * Gets the BooleanType object.
-   */
-  public static final BooleanType BooleanType = new BooleanType();
-
-  /**
-   * Gets the DateType object.
-   */
-  public static final DateType DateType = new DateType();
-
-  /**
-   * Gets the TimestampType object.
-   */
-  public static final TimestampType TimestampType = new TimestampType();
-
-  /**
-   * Gets the DoubleType object.
-   */
-  public static final DoubleType DoubleType = new DoubleType();
-
-  /**
-   * Gets the FloatType object.
-   */
-  public static final FloatType FloatType = new FloatType();
-
-  /**
-   * Gets the ByteType object.
-   */
-  public static final ByteType ByteType = new ByteType();
-
-  /**
-   * Gets the IntegerType object.
-   */
-  public static final IntegerType IntegerType = new IntegerType();
-
-  /**
-   * Gets the LongType object.
-   */
-  public static final LongType LongType = new LongType();
-
-  /**
-   * Gets the ShortType object.
-   */
-  public static final ShortType ShortType = new ShortType();
-
-  /**
-   * Gets the NullType object.
-   */
-  public static final NullType NullType = new NullType();
-
-  /**
-   * Creates an ArrayType by specifying the data type of elements ({@code elementType}).
-   * The field of {@code containsNull} is set to {@code true}.
-   */
-  public static ArrayType createArrayType(DataType elementType) {
-    if (elementType == null) {
-      throw new IllegalArgumentException("elementType should not be null.");
-    }
-
-    return new ArrayType(elementType, true);
-  }
-
-  /**
-   * Creates an ArrayType by specifying the data type of elements ({@code elementType}) and
-   * whether the array contains null values ({@code containsNull}).
-   */
-  public static ArrayType createArrayType(DataType elementType, boolean containsNull) {
-    if (elementType == null) {
-      throw new IllegalArgumentException("elementType should not be null.");
-    }
-
-    return new ArrayType(elementType, containsNull);
-  }
-
-  /**
-   * Creates a MapType by specifying the data type of keys ({@code keyType}) and values
-   * ({@code keyType}). The field of {@code valueContainsNull} is set to {@code true}.
-   */
-  public static MapType createMapType(DataType keyType, DataType valueType) {
-    if (keyType == null) {
-      throw new IllegalArgumentException("keyType should not be null.");
-    }
-    if (valueType == null) {
-      throw new IllegalArgumentException("valueType should not be null.");
-    }
-
-    return new MapType(keyType, valueType, true);
-  }
-
-  /**
-   * Creates a MapType by specifying the data type of keys ({@code keyType}), the data type of
-   * values ({@code keyType}), and whether values contain any null value
-   * ({@code valueContainsNull}).
-   */
-  public static MapType createMapType(
-      DataType keyType,
-      DataType valueType,
-      boolean valueContainsNull) {
-    if (keyType == null) {
-      throw new IllegalArgumentException("keyType should not be null.");
-    }
-    if (valueType == null) {
-      throw new IllegalArgumentException("valueType should not be null.");
-    }
-
-    return new MapType(keyType, valueType, valueContainsNull);
-  }
-
-  /**
-   * Creates a StructField by specifying the name ({@code name}), data type ({@code dataType}) and
-   * whether values of this field can be null values ({@code nullable}).
-   */
-  public static StructField createStructField(
-      String name,
-      DataType dataType,
-      boolean nullable,
-      Metadata metadata) {
-    if (name == null) {
-      throw new IllegalArgumentException("name should not be null.");
-    }
-    if (dataType == null) {
-      throw new IllegalArgumentException("dataType should not be null.");
-    }
-    if (metadata == null) {
-      throw new IllegalArgumentException("metadata should not be null.");
-    }
-
-    return new StructField(name, dataType, nullable, metadata);
-  }
-
-  /**
-   * Creates a StructField with empty metadata.
-   *
-   * @see #createStructField(String, DataType, boolean, Metadata)
-   */
-  public static StructField createStructField(String name, DataType dataType, boolean nullable) {
-    return createStructField(name, dataType, nullable, (new MetadataBuilder()).build());
-  }
-
-  /**
-   * Creates a StructType with the given list of StructFields ({@code fields}).
-   */
-  public static StructType createStructType(List<StructField> fields) {
-    return createStructType(fields.toArray(new StructField[0]));
-  }
-
-  /**
-   * Creates a StructType with the given StructField array ({@code fields}).
-   */
-  public static StructType createStructType(StructField[] fields) {
-    if (fields == null) {
-      throw new IllegalArgumentException("fields should not be null.");
-    }
-    Set<String> distinctNames = new HashSet<String>();
-    for (StructField field: fields) {
-      if (field == null) {
-        throw new IllegalArgumentException(
-          "fields should not contain any null.");
-      }
-
-      distinctNames.add(field.getName());
-    }
-    if (distinctNames.size() != fields.length) {
-      throw new IllegalArgumentException("fields should have distinct names.");
-    }
-
-    return new StructType(fields);
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java
deleted file mode 100644
index 6677793..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java
+++ /dev/null
@@ -1,27 +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.api.java;
-
-/**
- * The data type representing java.sql.Date values.
- *
- * {@code DateType} is represented by the singleton object {@link DataType#DateType}.
- */
-public class DateType extends DataType {
-    protected DateType() {}
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java
deleted file mode 100644
index 6075245..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java
+++ /dev/null
@@ -1,79 +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.api.java;
-
-/**
- * The data type representing java.math.BigDecimal values.
- */
-public class DecimalType extends DataType {
-  private boolean hasPrecisionInfo;
-  private int precision;
-  private int scale;
-
-  public DecimalType(int precision, int scale) {
-    this.hasPrecisionInfo = true;
-    this.precision = precision;
-    this.scale = scale;
-  }
-
-  public DecimalType() {
-    this.hasPrecisionInfo = false;
-    this.precision = -1;
-    this.scale = -1;
-  }
-
-  public boolean isUnlimited() {
-    return !hasPrecisionInfo;
-  }
-
-  public boolean isFixed() {
-    return hasPrecisionInfo;
-  }
-
-  /** Return the precision, or -1 if no precision is set */
-  public int getPrecision() {
-    return precision;
-  }
-
-  /** Return the scale, or -1 if no precision is set */
-  public int getScale() {
-    return scale;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    DecimalType that = (DecimalType) o;
-
-    if (hasPrecisionInfo != that.hasPrecisionInfo) return false;
-    if (precision != that.precision) return false;
-    if (scale != that.scale) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = (hasPrecisionInfo ? 1 : 0);
-    result = 31 * result + precision;
-    result = 31 * result + scale;
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java
deleted file mode 100644
index f0060d0..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java
+++ /dev/null
@@ -1,27 +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.api.java;
-
-/**
- * The data type representing double and Double values.
- *
- * {@code DoubleType} is represented by the singleton object {@link DataType#DoubleType}.
- */
-public class DoubleType extends DataType {
-  protected DoubleType() {}
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java
deleted file mode 100644
index 4a6a37f..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java
+++ /dev/null
@@ -1,27 +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.api.java;
-
-/**
- * The data type representing float and Float values.
- *
- * {@code FloatType} is represented by the singleton object {@link DataType#FloatType}.
- */
-public class FloatType extends DataType {
-  protected FloatType() {}
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java
deleted file mode 100644
index bfd7049..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java
+++ /dev/null
@@ -1,27 +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.api.java;
-
-/**
- * The data type representing int and Integer values.
- *
- * {@code IntegerType} is represented by the singleton object {@link DataType#IntegerType}.
- */
-public class IntegerType extends DataType {
-  protected IntegerType() {}
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java
deleted file mode 100644
index af13a46..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java
+++ /dev/null
@@ -1,27 +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.api.java;
-
-/**
- * The data type representing long and Long values.
- *
- * {@code LongType} is represented by the singleton object {@link DataType#LongType}.
- */
-public class LongType extends DataType {
-  protected LongType() {}
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java
deleted file mode 100644
index 063e6b3..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java
+++ /dev/null
@@ -1,78 +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.api.java;
-
-/**
- * The data type representing Maps. A MapType object comprises two fields,
- * {@code DataType keyType}, {@code DataType valueType}, and {@code boolean valueContainsNull}.
- * The field of {@code keyType} is used to specify the type of keys in the map.
- * The field of {@code valueType} is used to specify the type of values in the map.
- * The field of {@code valueContainsNull} is used to specify if map values have
- * {@code null} values.
- * For values of a MapType column, keys are not allowed to have {@code null} values.
- *
- * To create a {@link MapType},
- * {@link DataType#createMapType(DataType, DataType)} or
- * {@link DataType#createMapType(DataType, DataType, boolean)}
- * should be used.
- */
-public class MapType extends DataType {
-  private DataType keyType;
-  private DataType valueType;
-  private boolean valueContainsNull;
-
-  protected MapType(DataType keyType, DataType valueType, boolean valueContainsNull) {
-    this.keyType = keyType;
-    this.valueType = valueType;
-    this.valueContainsNull = valueContainsNull;
-  }
-
-  public DataType getKeyType() {
-    return keyType;
-  }
-
-  public DataType getValueType() {
-    return valueType;
-  }
-
-  public boolean isValueContainsNull() {
-    return valueContainsNull;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    MapType mapType = (MapType) o;
-
-    if (valueContainsNull != mapType.valueContainsNull) return false;
-    if (!keyType.equals(mapType.keyType)) return false;
-    if (!valueType.equals(mapType.valueType)) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = keyType.hashCode();
-    result = 31 * result + valueType.hashCode();
-    result = 31 * result + (valueContainsNull ? 1 : 0);
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java
deleted file mode 100644
index 0f819fb..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java
+++ /dev/null
@@ -1,31 +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.api.java;
-
-/**
- * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean,
- * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and
- * Array[Metadata]. JSON is used for serialization.
- *
- * The default constructor is private. User should use [[MetadataBuilder]].
- */
-class Metadata extends org.apache.spark.sql.catalyst.util.Metadata {
-  Metadata(scala.collection.immutable.Map<String, Object> map) {
-    super(map);
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java
deleted file mode 100644
index 6e6b12f..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java
+++ /dev/null
@@ -1,28 +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.api.java;
-
-/**
- * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former.
- */
-public class MetadataBuilder extends org.apache.spark.sql.catalyst.util.MetadataBuilder {
-  @Override
-  public Metadata build() {
-    return new Metadata(getMap());
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java
deleted file mode 100644
index 6d5ecdf..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java
+++ /dev/null
@@ -1,27 +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.api.java;
-
-/**
- * The data type representing null and NULL values.
- *
- * {@code NullType} is represented by the singleton object {@link DataType#NullType}.
- */
-public class NullType extends DataType {
-  protected NullType() {}
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java
deleted file mode 100644
index 7d7604b..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java
+++ /dev/null
@@ -1,27 +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.api.java;
-
-/**
- * The data type representing short and Short values.
- *
- * {@code ShortType} is represented by the singleton object {@link DataType#ShortType}.
- */
-public class ShortType extends DataType {
-  protected ShortType() {}
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java
deleted file mode 100644
index f4ba0c0..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java
+++ /dev/null
@@ -1,27 +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.api.java;
-
-/**
- * The data type representing String values.
- *
- * {@code StringType} is represented by the singleton object {@link DataType#StringType}.
- */
-public class StringType extends DataType {
-  protected StringType() {}
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java
deleted file mode 100644
index 7c60d49..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java
+++ /dev/null
@@ -1,91 +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.api.java;
-
-import java.util.Map;
-
-/**
- * A StructField object represents a field in a StructType object.
- * A StructField object comprises three fields, {@code String name}, {@code DataType dataType},
- * and {@code boolean nullable}. The field of {@code name} is the name of a StructField.
- * The field of {@code dataType} specifies the data type of a StructField.
- * The field of {@code nullable} specifies if values of a StructField can contain {@code null}
- * values.
- * The field of {@code metadata} provides extra information of the StructField.
- *
- * To create a {@link StructField},
- * {@link DataType#createStructField(String, DataType, boolean, Metadata)}
- * should be used.
- */
-public class StructField {
-  private String name;
-  private DataType dataType;
-  private boolean nullable;
-  private Metadata metadata;
-
-  protected StructField(
-      String name,
-      DataType dataType,
-      boolean nullable,
-      Metadata metadata) {
-    this.name = name;
-    this.dataType = dataType;
-    this.nullable = nullable;
-    this.metadata = metadata;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public DataType getDataType() {
-    return dataType;
-  }
-
-  public boolean isNullable() {
-    return nullable;
-  }
-
-  public Metadata getMetadata() {
-    return metadata;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    StructField that = (StructField) o;
-
-    if (nullable != that.nullable) return false;
-    if (!dataType.equals(that.dataType)) return false;
-    if (!name.equals(that.name)) return false;
-    if (!metadata.equals(that.metadata)) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = name.hashCode();
-    result = 31 * result + dataType.hashCode();
-    result = 31 * result + (nullable ? 1 : 0);
-    result = 31 * result + metadata.hashCode();
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java
deleted file mode 100644
index a4b501e..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.api.java;
-
-import java.util.Arrays;
-
-/**
- * The data type representing Rows.
- * A StructType object comprises an array of StructFields.
- *
- * To create an {@link StructType},
- * {@link DataType#createStructType(java.util.List)} or
- * {@link DataType#createStructType(StructField[])}
- * should be used.
- */
-public class StructType extends DataType {
-  private StructField[] fields;
-
-  protected StructType(StructField[] fields) {
-    this.fields = fields;
-  }
-
-  public StructField[] getFields() {
-    return fields;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    StructType that = (StructType) o;
-
-    if (!Arrays.equals(fields, that.fields)) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    return Arrays.hashCode(fields);
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java
deleted file mode 100644
index 06d44c7..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java
+++ /dev/null
@@ -1,27 +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.api.java;
-
-/**
- * The data type representing java.sql.Timestamp values.
- *
- * {@code TimestampType} is represented by the singleton object {@link DataType#TimestampType}.
- */
-public class TimestampType extends DataType {
-  protected TimestampType() {}
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java
deleted file mode 100644
index f0d079d..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java
+++ /dev/null
@@ -1,54 +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.api.java;
-
-import java.io.Serializable;
-
-import org.apache.spark.annotation.DeveloperApi;
-
-/**
- * ::DeveloperApi::
- * The data type representing User-Defined Types (UDTs).
- * UDTs may use any other DataType for an underlying representation.
- */
-@DeveloperApi
-public abstract class UserDefinedType<UserType> extends DataType implements Serializable {
-
-  protected UserDefinedType() { }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-    @SuppressWarnings("unchecked")
-    UserDefinedType<UserType> that = (UserDefinedType<UserType>) o;
-    return this.sqlType().equals(that.sqlType());
-  }
-
-  /** Underlying storage type for this UDT */
-  public abstract DataType sqlType();
-
-  /** Convert the user type to a SQL datum */
-  public abstract Object serialize(Object obj);
-
-  /** Convert a SQL datum to the user type */
-  public abstract UserType deserialize(Object datum);
-
-  /** Class object for the UserType */
-  public abstract Class<UserType> userClass();
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 d8efce0..d9f3b3a 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
@@ -24,7 +24,6 @@ import scala.language.implicitConversions
 import scala.reflect.runtime.universe.TypeTag
 
 import org.apache.hadoop.conf.Configuration
-
 import org.apache.spark.SparkContext
 import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental}
 import org.apache.spark.rdd.RDD
@@ -32,14 +31,14 @@ 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.{Optimizer, DefaultOptimizer}
+import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
-import org.apache.spark.sql.catalyst.types.UserDefinedType
-import org.apache.spark.sql.execution.{SparkStrategies, _}
+import org.apache.spark.sql.execution._
 import org.apache.spark.sql.json._
 import org.apache.spark.sql.parquet.ParquetRelation
-import org.apache.spark.sql.sources.{DataSourceStrategy, BaseRelation, DDLParser, LogicalRelation}
+import org.apache.spark.sql.sources.{BaseRelation, DDLParser, DataSourceStrategy, LogicalRelation}
+import org.apache.spark.sql.types._
 
 /**
  * :: AlphaComponent ::

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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
index 80787b6..686bcdf 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala
@@ -17,8 +17,7 @@
 
 package org.apache.spark.sql
 
-import java.util.{Map => JMap, List => JList}
-
+import java.util.{List => JList}
 
 import scala.collection.JavaConversions._
 
@@ -37,8 +36,9 @@ 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.json.JsonRDD
 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
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
index 65358b7..f10ee7b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
@@ -17,12 +17,14 @@
 
 package org.apache.spark.sql
 
+import scala.util.parsing.combinator.RegexParsers
+
 import org.apache.spark.sql.catalyst.{SqlLexical, AbstractSparkSQLParser}
 import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.{UncacheTableCommand, CacheTableCommand, SetCommand}
+import org.apache.spark.sql.types.StringType
 
-import scala.util.parsing.combinator.RegexParsers
 
 /**
  * The top level Spark SQL parser. This parser recognizes syntaxes that are available for all SQL

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
index 7f868cd..a75f559 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
@@ -23,15 +23,13 @@ import org.apache.hadoop.conf.Configuration
 
 import org.apache.spark.annotation.{DeveloperApi, Experimental}
 import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
-import org.apache.spark.sql.{SQLContext, StructType => SStructType}
-import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType
+import org.apache.spark.sql.SQLContext
 import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow}
 import org.apache.spark.sql.execution.LogicalRDD
 import org.apache.spark.sql.json.JsonRDD
 import org.apache.spark.sql.parquet.ParquetRelation
 import org.apache.spark.sql.sources.{LogicalRelation, BaseRelation}
-import org.apache.spark.sql.types.util.DataTypeConversions
-import org.apache.spark.sql.types.util.DataTypeConversions.asScalaDataType
+import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
 
 /**
@@ -126,9 +124,8 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration {
   @DeveloperApi
   def applySchema(rowRDD: JavaRDD[Row], schema: StructType): JavaSchemaRDD = {
     val scalaRowRDD = rowRDD.rdd.map(r => r.row)
-    val scalaSchema = asScalaDataType(schema).asInstanceOf[SStructType]
     val logicalPlan =
-      LogicalRDD(scalaSchema.toAttributes, scalaRowRDD)(sqlContext)
+      LogicalRDD(schema.toAttributes, scalaRowRDD)(sqlContext)
     new JavaSchemaRDD(sqlContext, logicalPlan)
   }
 
@@ -184,10 +181,10 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration {
   def jsonRDD(json: JavaRDD[String], schema: StructType): JavaSchemaRDD = {
     val columnNameOfCorruptJsonRecord = sqlContext.conf.columnNameOfCorruptRecord
     val appliedScalaSchema =
-      Option(asScalaDataType(schema)).getOrElse(
+      Option(schema).getOrElse(
         JsonRDD.nullTypeToStringType(
           JsonRDD.inferSchema(
-            json.rdd, 1.0, columnNameOfCorruptJsonRecord))).asInstanceOf[SStructType]
+            json.rdd, 1.0, columnNameOfCorruptJsonRecord)))
     val scalaRowRDD = JsonRDD.jsonStringToRow(
       json.rdd, appliedScalaSchema, columnNameOfCorruptJsonRecord)
     val logicalPlan =
@@ -218,43 +215,25 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration {
       val (dataType, nullable) = property.getPropertyType match {
         case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) =>
           (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true)
-        case c: Class[_] if c == classOf[java.lang.String] =>
-          (org.apache.spark.sql.StringType, true)
-        case c: Class[_] if c == java.lang.Short.TYPE =>
-          (org.apache.spark.sql.ShortType, false)
-        case c: Class[_] if c == java.lang.Integer.TYPE =>
-          (org.apache.spark.sql.IntegerType, false)
-        case c: Class[_] if c == java.lang.Long.TYPE =>
-          (org.apache.spark.sql.LongType, false)
-        case c: Class[_] if c == java.lang.Double.TYPE =>
-          (org.apache.spark.sql.DoubleType, false)
-        case c: Class[_] if c == java.lang.Byte.TYPE =>
-          (org.apache.spark.sql.ByteType, false)
-        case c: Class[_] if c == java.lang.Float.TYPE =>
-          (org.apache.spark.sql.FloatType, false)
-        case c: Class[_] if c == java.lang.Boolean.TYPE =>
-          (org.apache.spark.sql.BooleanType, false)
-
-        case c: Class[_] if c == classOf[java.lang.Short] =>
-          (org.apache.spark.sql.ShortType, true)
-        case c: Class[_] if c == classOf[java.lang.Integer] =>
-          (org.apache.spark.sql.IntegerType, true)
-        case c: Class[_] if c == classOf[java.lang.Long] =>
-          (org.apache.spark.sql.LongType, true)
-        case c: Class[_] if c == classOf[java.lang.Double] =>
-          (org.apache.spark.sql.DoubleType, true)
-        case c: Class[_] if c == classOf[java.lang.Byte] =>
-          (org.apache.spark.sql.ByteType, true)
-        case c: Class[_] if c == classOf[java.lang.Float] =>
-          (org.apache.spark.sql.FloatType, true)
-        case c: Class[_] if c == classOf[java.lang.Boolean] =>
-          (org.apache.spark.sql.BooleanType, true)
-        case c: Class[_] if c == classOf[java.math.BigDecimal] =>
-          (org.apache.spark.sql.DecimalType(), true)
-        case c: Class[_] if c == classOf[java.sql.Date] =>
-          (org.apache.spark.sql.DateType, true)
-        case c: Class[_] if c == classOf[java.sql.Timestamp] =>
-          (org.apache.spark.sql.TimestampType, true)
+        case c: Class[_] if c == classOf[java.lang.String] => (StringType, true)
+        case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false)
+        case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false)
+        case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false)
+        case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false)
+        case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false)
+        case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false)
+        case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false)
+
+        case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true)
+        case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true)
+        case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true)
+        case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true)
+        case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true)
+        case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true)
+        case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true)
+        case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType(), true)
+        case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true)
+        case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true)
       }
       AttributeReference(property.getName, dataType, nullable)()
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
index 5b9c612..9e10e53 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
@@ -20,13 +20,12 @@ package org.apache.spark.sql.api.java
 import java.util.{List => JList}
 
 import org.apache.spark.Partitioner
-import org.apache.spark.api.java.{JavaRDDLike, JavaRDD}
+import org.apache.spark.api.java.{JavaRDD, JavaRDDLike}
 import org.apache.spark.api.java.function.{Function => JFunction}
-import org.apache.spark.sql.types.util.DataTypeConversions
+import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.{SQLContext, SchemaRDD, SchemaRDDLike}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import DataTypeConversions._
-import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.types.StructType
 import org.apache.spark.storage.StorageLevel
 
 /**
@@ -59,8 +58,7 @@ class JavaSchemaRDD(
   override def toString: String = baseSchemaRDD.toString
 
   /** Returns the schema of this JavaSchemaRDD (represented by a StructType). */
-  def schema: StructType =
-    asJavaDataType(baseSchemaRDD.schema).asInstanceOf[StructType]
+  def schema: StructType = baseSchemaRDD.schema.asInstanceOf[StructType]
 
   // =======================================================================
   // Base RDD functions that do NOT change schema

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala
index 401798e..207e280 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala
@@ -17,8 +17,6 @@
 
 package org.apache.spark.sql.api.java
 
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
-
 import scala.annotation.varargs
 import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper}
 import scala.collection.JavaConversions


---------------------------------------------------------------------
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-5123][SQL] Reconcile Java/Scala API for data types.

Posted by rx...@apache.org.
[SPARK-5123][SQL] Reconcile Java/Scala API for data types.

Having two versions of the data type APIs (one for Java, one for Scala) requires downstream libraries to also have two versions of the APIs if the library wants to support both Java and Scala. I took a look at the Scala version of the data type APIs - it can actually work out pretty well for Java out of the box.

As part of the PR, I created a sql.types package and moved all type definitions there. I then removed the Java specific data type API along with a lot of the conversion code.

This subsumes https://github.com/apache/spark/pull/3925

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

Closes #3958 from rxin/SPARK-5123-datatype-2 and squashes the following commits:

66505cc [Reynold Xin] [SPARK-5123] Expose only one version of the data type APIs (i.e. remove the Java-specific API).


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

Branch: refs/heads/master
Commit: f9969098c8cb15e36c718b80c6cf5b534a6cf7c3
Parents: 14e3f11
Author: Reynold Xin <rx...@databricks.com>
Authored: Tue Jan 13 17:16:41 2015 -0800
Committer: Reynold Xin <rx...@databricks.com>
Committed: Tue Jan 13 17:16:41 2015 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/ml/Pipeline.scala    |   5 +-
 .../scala/org/apache/spark/ml/Transformer.scala |   2 +-
 .../ml/classification/LogisticRegression.scala  |   1 +
 .../BinaryClassificationEvaluator.scala         |   3 +-
 .../org/apache/spark/ml/feature/HashingTF.scala |   2 +-
 .../spark/ml/feature/StandardScaler.scala       |   1 +
 .../org/apache/spark/ml/feature/Tokenizer.scala |   2 +-
 .../apache/spark/ml/tuning/CrossValidator.scala |   3 +-
 .../org/apache/spark/mllib/linalg/Vectors.scala |   3 +-
 project/MimaExcludes.scala                      |  12 +
 project/SparkBuild.scala                        |   4 +-
 sql/README.md                                   |   2 +-
 .../spark/sql/catalyst/ScalaReflection.scala    |   5 +-
 .../apache/spark/sql/catalyst/SqlParser.scala   |   2 +-
 .../spark/sql/catalyst/analysis/Analyzer.scala  |   4 +-
 .../catalyst/analysis/HiveTypeCoercion.scala    |   2 +-
 .../catalyst/annotation/SQLUserDefinedType.java |  46 -
 .../apache/spark/sql/catalyst/dsl/package.scala |   4 +-
 .../catalyst/expressions/BoundAttribute.scala   |   2 +-
 .../spark/sql/catalyst/expressions/Cast.scala   |   4 +-
 .../sql/catalyst/expressions/Expression.scala   |   3 +-
 .../spark/sql/catalyst/expressions/Rand.scala   |   2 +-
 .../spark/sql/catalyst/expressions/Row.scala    |   2 +-
 .../sql/catalyst/expressions/ScalaUdf.scala     |   3 +-
 .../expressions/SpecificMutableRow.scala        |   2 +-
 .../sql/catalyst/expressions/WrapDynamic.scala  |   2 +-
 .../sql/catalyst/expressions/aggregates.scala   |   2 +-
 .../sql/catalyst/expressions/arithmetic.scala   |   2 +-
 .../expressions/codegen/CodeGenerator.scala     |  10 +-
 .../expressions/codegen/GenerateOrdering.scala  |   2 +-
 .../codegen/GenerateProjection.scala            |   2 +-
 .../sql/catalyst/expressions/complexTypes.scala |   2 +-
 .../catalyst/expressions/decimalFunctions.scala |   4 +-
 .../sql/catalyst/expressions/generators.scala   |   2 +-
 .../sql/catalyst/expressions/literals.scala     |   4 +-
 .../catalyst/expressions/namedExpressions.scala |   3 +-
 .../sql/catalyst/expressions/predicates.scala   |   2 +-
 .../spark/sql/catalyst/expressions/sets.scala   |   2 +-
 .../catalyst/expressions/stringOperations.scala |   2 +-
 .../sql/catalyst/optimizer/Optimizer.scala      |   4 +-
 .../org/apache/spark/sql/catalyst/package.scala |   2 +-
 .../spark/sql/catalyst/plans/QueryPlan.scala    |   2 +-
 .../catalyst/plans/logical/LogicalPlan.scala    |  17 +-
 .../catalyst/plans/logical/basicOperators.scala |   2 +-
 .../catalyst/plans/physical/partitioning.scala  |   2 +-
 .../spark/sql/catalyst/types/dataTypes.scala    | 650 --------------
 .../sql/catalyst/types/decimal/Decimal.scala    | 335 -------
 .../spark/sql/catalyst/types/package.scala      |  23 -
 .../spark/sql/catalyst/util/Metadata.scala      | 258 ------
 .../spark/sql/types/DataTypeConversions.scala   |  68 ++
 .../org/apache/spark/sql/types/DataTypes.java   | 208 +++++
 .../org/apache/spark/sql/types/Metadata.scala   | 268 ++++++
 .../spark/sql/types/SQLUserDefinedType.java     |  45 +
 .../org/apache/spark/sql/types/dataTypes.scala  | 900 +++++++++++++++++++
 .../spark/sql/types/decimal/Decimal.scala       | 335 +++++++
 .../org/apache/spark/sql/types/package.scala    |  24 +
 .../sql/catalyst/ScalaReflectionSuite.scala     |   2 +-
 .../sql/catalyst/analysis/AnalysisSuite.scala   |   2 +-
 .../analysis/DecimalPrecisionSuite.scala        |   2 +-
 .../analysis/HiveTypeCoercionSuite.scala        |   2 +-
 .../expressions/ExpressionEvaluationSuite.scala |  10 +-
 .../optimizer/ConstantFoldingSuite.scala        |   2 +-
 .../catalyst/optimizer/OptimizeInSuite.scala    |   2 +-
 .../sql/catalyst/trees/TreeNodeSuite.scala      |   2 +-
 .../catalyst/types/decimal/DecimalSuite.scala   | 157 ----
 .../spark/sql/catalyst/util/MetadataSuite.scala |   2 +
 .../apache/spark/sql/types/DataTypeSuite.scala  |  88 ++
 .../spark/sql/types/decimal/DecimalSuite.scala  | 157 ++++
 .../apache/spark/sql/api/java/ArrayType.java    |  68 --
 .../apache/spark/sql/api/java/BinaryType.java   |  27 -
 .../apache/spark/sql/api/java/BooleanType.java  |  27 -
 .../org/apache/spark/sql/api/java/ByteType.java |  27 -
 .../org/apache/spark/sql/api/java/DataType.java | 208 -----
 .../org/apache/spark/sql/api/java/DateType.java |  27 -
 .../apache/spark/sql/api/java/DecimalType.java  |  79 --
 .../apache/spark/sql/api/java/DoubleType.java   |  27 -
 .../apache/spark/sql/api/java/FloatType.java    |  27 -
 .../apache/spark/sql/api/java/IntegerType.java  |  27 -
 .../org/apache/spark/sql/api/java/LongType.java |  27 -
 .../org/apache/spark/sql/api/java/MapType.java  |  78 --
 .../org/apache/spark/sql/api/java/Metadata.java |  31 -
 .../spark/sql/api/java/MetadataBuilder.java     |  28 -
 .../org/apache/spark/sql/api/java/NullType.java |  27 -
 .../apache/spark/sql/api/java/ShortType.java    |  27 -
 .../apache/spark/sql/api/java/StringType.java   |  27 -
 .../apache/spark/sql/api/java/StructField.java  |  91 --
 .../apache/spark/sql/api/java/StructType.java   |  58 --
 .../spark/sql/api/java/TimestampType.java       |  27 -
 .../spark/sql/api/java/UserDefinedType.java     |  54 --
 .../scala/org/apache/spark/sql/SQLContext.scala |   9 +-
 .../scala/org/apache/spark/sql/SchemaRDD.scala  |   6 +-
 .../org/apache/spark/sql/SparkSQLParser.scala   |   4 +-
 .../spark/sql/api/java/JavaSQLContext.scala     |  69 +-
 .../spark/sql/api/java/JavaSchemaRDD.scala      |  10 +-
 .../org/apache/spark/sql/api/java/Row.scala     |   2 -
 .../spark/sql/api/java/UDFRegistration.scala    | 139 ++-
 .../apache/spark/sql/api/java/UDTWrappers.scala |  75 --
 .../spark/sql/columnar/ColumnAccessor.scala     |   4 +-
 .../spark/sql/columnar/ColumnBuilder.scala      |   2 +-
 .../apache/spark/sql/columnar/ColumnStats.scala |   2 +-
 .../apache/spark/sql/columnar/ColumnType.scala  |   2 +-
 .../CompressibleColumnAccessor.scala            |   2 +-
 .../compression/CompressibleColumnBuilder.scala |   2 +-
 .../compression/CompressionScheme.scala         |   2 +-
 .../compression/compressionSchemes.scala        |   3 +-
 .../spark/sql/execution/ExistingRDD.scala       |   3 +-
 .../sql/execution/GeneratedAggregate.scala      |   2 +-
 .../sql/execution/SparkSqlSerializer.scala      |   2 +-
 .../spark/sql/execution/SparkStrategies.scala   |   6 +-
 .../spark/sql/execution/debug/package.scala     |   2 +-
 .../apache/spark/sql/execution/pythonUdfs.scala |   4 +-
 .../apache/spark/sql/json/JSONRelation.scala    |   3 +-
 .../org/apache/spark/sql/json/JsonRDD.scala     |   8 +-
 .../scala/org/apache/spark/sql/package.scala    | 349 -------
 .../spark/sql/parquet/ParquetConverter.scala    |  17 +-
 .../spark/sql/parquet/ParquetFilters.scala      |   2 +-
 .../spark/sql/parquet/ParquetTableSupport.scala |   4 +-
 .../apache/spark/sql/parquet/ParquetTypes.scala |  16 +-
 .../apache/spark/sql/parquet/newParquet.scala   |   7 +-
 .../spark/sql/sources/LogicalRelation.scala     |   4 +-
 .../org/apache/spark/sql/sources/ddl.scala      |  10 +-
 .../apache/spark/sql/sources/interfaces.scala   |   3 +-
 .../apache/spark/sql/test/ExamplePointUDT.scala |   4 +-
 .../sql/types/util/DataTypeConversions.scala    | 175 ----
 .../apache/spark/sql/api/java/JavaAPISuite.java |   9 +-
 .../sql/api/java/JavaApplySchemaSuite.java      |  23 +-
 .../java/JavaSideDataTypeConversionSuite.java   | 150 ----
 .../org/apache/spark/sql/DataTypeSuite.scala    |  88 --
 .../org/apache/spark/sql/DslQuerySuite.scala    |   1 +
 .../scala/org/apache/spark/sql/RowSuite.scala   |   1 +
 .../org/apache/spark/sql/SQLQuerySuite.scala    |   3 +-
 .../sql/ScalaReflectionRelationSuite.scala      |   1 -
 .../apache/spark/sql/UserDefinedTypeSuite.scala |   3 +-
 .../spark/sql/api/java/JavaSQLSuite.scala       |   9 +-
 .../java/ScalaSideDataTypeConversionSuite.scala |  89 --
 .../spark/sql/columnar/ColumnStatsSuite.scala   |   2 +-
 .../spark/sql/columnar/ColumnTypeSuite.scala    |   2 +-
 .../spark/sql/columnar/ColumnarTestUtils.scala  |   2 +-
 .../columnar/NullableColumnAccessorSuite.scala  |   2 +-
 .../columnar/NullableColumnBuilderSuite.scala   |   2 +-
 .../compression/DictionaryEncodingSuite.scala   |   2 +-
 .../compression/IntegralDeltaSuite.scala        |   2 +-
 .../compression/RunLengthEncodingSuite.scala    |   2 +-
 .../TestCompressibleColumnBuilder.scala         |   2 +-
 .../org/apache/spark/sql/json/JsonSuite.scala   |  16 +-
 .../spark/sql/parquet/ParquetIOSuite.scala      |   6 +-
 .../spark/sql/parquet/ParquetQuerySuite.scala   |  16 +-
 .../spark/sql/parquet/ParquetSchemaSuite.scala  |  21 +-
 .../spark/sql/sources/FilteredScanSuite.scala   |   2 +
 .../spark/sql/sources/PrunedScanSuite.scala     |   1 +
 .../spark/sql/sources/TableScanSuite.scala      |   2 +-
 .../spark/sql/hive/thriftserver/Shim12.scala    |   4 +-
 .../spark/sql/hive/thriftserver/Shim13.scala    |   4 +-
 .../org/apache/spark/sql/hive/HiveContext.scala |   2 +-
 .../apache/spark/sql/hive/HiveInspectors.scala  |  15 +-
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |   2 +-
 .../org/apache/spark/sql/hive/HiveQl.scala      |   5 +-
 .../apache/spark/sql/hive/HiveStrategies.scala  |   7 +-
 .../sql/hive/execution/HiveNativeCommand.scala  |   2 +-
 .../sql/hive/execution/HiveTableScan.scala      |   4 +-
 .../spark/sql/hive/execution/commands.scala     |   4 +-
 .../org/apache/spark/sql/hive/hiveUdfs.scala    |   2 +-
 .../spark/sql/hive/HiveInspectorSuite.scala     |  19 +-
 .../sql/hive/HiveMetastoreCatalogSuite.scala    |   2 +-
 .../sql/hive/InsertIntoHiveTableSuite.scala     |   1 +
 .../sql/hive/MetastoreDataSourcesSuite.scala    |   1 +
 .../org/apache/spark/sql/hive/Shim12.scala      |  20 +-
 .../org/apache/spark/sql/hive/Shim13.scala      |  11 +-
 168 files changed, 2465 insertions(+), 3805 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 081a574..ad6fed1 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
@@ -21,8 +21,9 @@ import scala.collection.mutable.ListBuffer
 
 import org.apache.spark.Logging
 import org.apache.spark.annotation.AlphaComponent
-import org.apache.spark.ml.param.{Params, Param, ParamMap}
-import org.apache.spark.sql.{SchemaRDD, StructType}
+import org.apache.spark.ml.param.{Param, ParamMap}
+import org.apache.spark.sql.SchemaRDD
+import org.apache.spark.sql.types.StructType
 
 /**
  * :: AlphaComponent ::

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 23fbd22..1331b91 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.SchemaRDD
 import org.apache.spark.sql.api.java.JavaSchemaRDD
 import org.apache.spark.sql.catalyst.analysis.Star
 import org.apache.spark.sql.catalyst.expressions.ScalaUdf
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 /**
  * :: AlphaComponent ::

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 85b8899..8c57081 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
@@ -26,6 +26,7 @@ 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.catalyst.dsl._
+import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
 import org.apache.spark.storage.StorageLevel
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 0b0504e..12473cb 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,8 @@ 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.{DoubleType, Row, SchemaRDD}
+import org.apache.spark.sql.{Row, SchemaRDD}
+import org.apache.spark.sql.types.DoubleType
 
 /**
  * :: AlphaComponent ::

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala
index e0bfb1e..0956062 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala
@@ -22,7 +22,7 @@ import org.apache.spark.ml.UnaryTransformer
 import org.apache.spark.ml.param.{IntParam, ParamMap}
 import org.apache.spark.mllib.feature
 import org.apache.spark.mllib.linalg.{VectorUDT, Vector}
-import org.apache.spark.sql.catalyst.types.DataType
+import org.apache.spark.sql.types.DataType
 
 /**
  * :: AlphaComponent ::

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 896a6b8..72825f6 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
@@ -25,6 +25,7 @@ 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.catalyst.dsl._
+import org.apache.spark.sql.types.{StructField, StructType}
 
 /**
  * Params for [[StandardScaler]] and [[StandardScalerModel]].

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala
index 9352f40..e622a5c 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala
@@ -20,7 +20,7 @@ package org.apache.spark.ml.feature
 import org.apache.spark.annotation.AlphaComponent
 import org.apache.spark.ml.UnaryTransformer
 import org.apache.spark.ml.param.ParamMap
-import org.apache.spark.sql.{DataType, StringType, ArrayType}
+import org.apache.spark.sql.types.{DataType, StringType, ArrayType}
 
 /**
  * :: AlphaComponent ::

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 194b9bf..08fe991 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,8 @@ 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, StructType}
+import org.apache.spark.sql.SchemaRDD
+import org.apache.spark.sql.types.StructType
 
 /**
  * Params for [[CrossValidator]] and [[CrossValidatorModel]].

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
index bf1faa2..adbd826 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
@@ -27,9 +27,8 @@ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV}
 
 import org.apache.spark.SparkException
 import org.apache.spark.mllib.util.NumericParser
-import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType
 import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Row}
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 /**
  * Represents a numeric vector, whose index type is Int and value type is Double.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/project/MimaExcludes.scala
----------------------------------------------------------------------
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 51e8bd4..f6f9f49 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -60,6 +60,18 @@ object MimaExcludes {
             ProblemFilters.exclude[IncompatibleResultTypeProblem](
               "org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler." +
                 "removeAndGetProcessor")
+          ) ++ Seq(
+            // SPARK-5123 (SparkSQL data type change) - alpha component only
+            ProblemFilters.exclude[IncompatibleResultTypeProblem](
+              "org.apache.spark.ml.feature.HashingTF.outputDataType"),
+            ProblemFilters.exclude[IncompatibleResultTypeProblem](
+              "org.apache.spark.ml.feature.Tokenizer.outputDataType"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem](
+              "org.apache.spark.ml.feature.Tokenizer.validateInputType"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem](
+              "org.apache.spark.ml.classification.LogisticRegressionModel.validateAndTransformSchema"),
+            ProblemFilters.exclude[IncompatibleMethTypeProblem](
+              "org.apache.spark.ml.classification.LogisticRegression.validateAndTransformSchema")
           )
 
         case v if v.startsWith("1.2") =>

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 46a54c6..b2c546d 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -254,10 +254,10 @@ object SQL {
         |import org.apache.spark.sql.catalyst.expressions._
         |import org.apache.spark.sql.catalyst.plans.logical._
         |import org.apache.spark.sql.catalyst.rules._
-        |import org.apache.spark.sql.catalyst.types._
         |import org.apache.spark.sql.catalyst.util._
         |import org.apache.spark.sql.execution
         |import org.apache.spark.sql.test.TestSQLContext._
+        |import org.apache.spark.sql.types._
         |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin,
     cleanupCommands in console := "sparkContext.stop()"
   )
@@ -284,11 +284,11 @@ object Hive {
         |import org.apache.spark.sql.catalyst.expressions._
         |import org.apache.spark.sql.catalyst.plans.logical._
         |import org.apache.spark.sql.catalyst.rules._
-        |import org.apache.spark.sql.catalyst.types._
         |import org.apache.spark.sql.catalyst.util._
         |import org.apache.spark.sql.execution
         |import org.apache.spark.sql.hive._
         |import org.apache.spark.sql.hive.test.TestHive._
+        |import org.apache.spark.sql.types._
         |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin,
     cleanupCommands in console := "sparkContext.stop()",
     // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/README.md
----------------------------------------------------------------------
diff --git a/sql/README.md b/sql/README.md
index 8d2f3cf..d058a6b 100644
--- a/sql/README.md
+++ b/sql/README.md
@@ -34,11 +34,11 @@ import org.apache.spark.sql.catalyst.errors._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.catalyst.util._
 import org.apache.spark.sql.execution
 import org.apache.spark.sql.hive._
 import org.apache.spark.sql.hive.TestHive._
+import org.apache.spark.sql.types._
 Welcome to Scala version 2.10.4 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45).
 Type in expressions to have them evaluated.
 Type :help for more information.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
index 2cf241d..d169da6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
@@ -20,11 +20,10 @@ package org.apache.spark.sql.catalyst
 import java.sql.{Date, Timestamp}
 
 import org.apache.spark.util.Utils
-import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType
 import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, AttributeReference, Row}
 import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
 
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
index 5d974df..d19563e 100755
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 /**
  * A very simple SQL parser.  Based loosely on:

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index c009cc1..bd00ff2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -22,8 +22,8 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.catalyst.types.StructType
-import org.apache.spark.sql.catalyst.types.IntegerType
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.types.IntegerType
 
 /**
  * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
index 242f28f..1535336 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 object HiveTypeCoercion {
   // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java
deleted file mode 100644
index e966aee..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java
+++ /dev/null
@@ -1,46 +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.catalyst.annotation;
-
-import java.lang.annotation.*;
-
-import org.apache.spark.annotation.DeveloperApi;
-import org.apache.spark.sql.catalyst.types.UserDefinedType;
-
-/**
- * ::DeveloperApi::
- * A user-defined type which can be automatically recognized by a SQLContext and registered.
- *
- * WARNING: This annotation will only work if both Java and Scala reflection return the same class
- *          names (after erasure) for the UDT.  This will NOT be the case when, e.g., the UDT class
- *          is enclosed in an object (a singleton).
- *
- * WARNING: UDTs are currently only supported from Scala.
- */
-// TODO: Should I used @Documented ?
-@DeveloperApi
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-public @interface SQLUserDefinedType {
-
-  /**
-   * Returns an instance of the UserDefinedType which can serialize and deserialize the user
-   * class to and from Catalyst built-in types.
-   */
-  Class<? extends UserDefinedType<?> > udt();
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
index b2262e5..bdac750 100755
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ -26,8 +26,8 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
 
 /**
  * A collection of implicit conversions that create a DSL for constructing catalyst data structures.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
index fa80b07..76a9f08 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
 
 import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.errors.attachTree
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 import org.apache.spark.sql.catalyst.trees
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index 4ede0b4..00961f0 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -22,8 +22,8 @@ import java.text.{DateFormat, SimpleDateFormat}
 
 import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.errors.TreeNodeException
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
 
 /** Cast the child expression to the target data type. */
 case class Cast(child: Expression, dataType: DataType) extends UnaryExpression with Logging {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index ac5b02c..cf14992 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -20,8 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
 import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.trees
 import org.apache.spark.sql.catalyst.trees.TreeNode
-import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType, NativeType}
-import org.apache.spark.sql.catalyst.util.Metadata
+import org.apache.spark.sql.types._
 
 abstract class Expression extends TreeNode[Expression] {
   self: Product =>

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
index 851db95..b2c6d30 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.catalyst.expressions
 
 import java.util.Random
-import org.apache.spark.sql.catalyst.types.DoubleType
+import org.apache.spark.sql.types.DoubleType
 
 
 case object Rand extends LeafExpression {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
index 463f366..dcda53b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.expressions
 
-import org.apache.spark.sql.catalyst.types.NativeType
+import org.apache.spark.sql.types.NativeType
 
 object Row {
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
index 18c96da..8a36c68 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
@@ -18,8 +18,7 @@
 package org.apache.spark.sql.catalyst.expressions
 
 import org.apache.spark.sql.catalyst.ScalaReflection
-import org.apache.spark.sql.catalyst.types.DataType
-import org.apache.spark.util.ClosureCleaner
+import org.apache.spark.sql.types.DataType
 
 /**
  * User-defined function.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
index 570379c..37d9f0e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.expressions
 
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 /**
  * A parent class for mutable container objects that are reused when the values are changed,

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala
index 1a4ac06..8328278 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
 
 import scala.language.dynamics
 
-import org.apache.spark.sql.catalyst.types.DataType
+import org.apache.spark.sql.types.DataType
 
 /**
  * The data type representing [[DynamicRow]] values.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
index 5ea9868..735b748 100755
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
 
 import com.clearspring.analytics.stream.cardinality.HyperLogLog
 
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 import org.apache.spark.sql.catalyst.trees
 import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.util.collection.OpenHashSet

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index 168a963..574907f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.catalyst.expressions
 
 import org.apache.spark.sql.catalyst.analysis.UnresolvedException
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 case class UnaryMinus(child: Expression) extends UnaryExpression {
   type EvaluatedType = Any

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
index 90c81b2..a5d6423 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
@@ -18,14 +18,14 @@
 package org.apache.spark.sql.catalyst.expressions.codegen
 
 import com.google.common.cache.{CacheLoader, CacheBuilder}
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.apache.spark.sql.types.decimal.Decimal
 
 import scala.language.existentials
 
 import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.expressions
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 // These classes are here to avoid issues with serialization and integration with quasiquotes.
 class IntegerHashSet extends org.apache.spark.util.collection.OpenHashSet[Int]
@@ -541,11 +541,11 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin
         childEval.code ++
         q"""
          var $nullTerm = ${childEval.nullTerm}
-         var $primitiveTerm: org.apache.spark.sql.catalyst.types.decimal.Decimal =
+         var $primitiveTerm: org.apache.spark.sql.types.decimal.Decimal =
            ${defaultPrimitive(DecimalType())}
 
          if (!$nullTerm) {
-           $primitiveTerm = new org.apache.spark.sql.catalyst.types.decimal.Decimal()
+           $primitiveTerm = new org.apache.spark.sql.types.decimal.Decimal()
            $primitiveTerm = $primitiveTerm.setOrNull(${childEval.primitiveTerm}, $precision, $scale)
            $nullTerm = $primitiveTerm == null
          }
@@ -627,7 +627,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin
     case LongType => ru.Literal(Constant(1L))
     case ByteType => ru.Literal(Constant(-1.toByte))
     case DoubleType => ru.Literal(Constant(-1.toDouble))
-    case DecimalType() => q"org.apache.spark.sql.catalyst.types.decimal.Decimal(-1)"
+    case DecimalType() => q"org.apache.spark.sql.types.decimal.Decimal(-1)"
     case IntegerType => ru.Literal(Constant(-1))
     case _ => ru.Literal(Constant(null))
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
index 094ff14..0db29eb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen
 
 import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.types.{StringType, NumericType}
+import org.apache.spark.sql.types.{StringType, NumericType}
 
 /**
  * Generates bytecode for an [[Ordering]] of [[Row Rows]] for a given set of

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala
index 2ff6116..cc97cb4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.catalyst.expressions.codegen
 
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
index 9aec601..1bc34f7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
 
 import scala.collection.Map
 
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 /**
  * Returns the item at `ordinal` in the Array `child` or the Key `ordinal` in Map `child`.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala
index d1eab2e..e54cfa1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala
@@ -17,8 +17,8 @@
 
 package org.apache.spark.sql.catalyst.expressions
 
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
-import org.apache.spark.sql.catalyst.types.{DecimalType, LongType, DoubleType, DataType}
+import org.apache.spark.sql.types.decimal.Decimal
+import org.apache.spark.sql.types.{DecimalType, LongType, DoubleType, DataType}
 
 /** Return the unscaled Long value of a Decimal, assuming it fits in a Long */
 case class UnscaledValue(child: Expression) extends UnaryExpression {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
index ab0701f..43b6482 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
 import scala.collection.Map
 
 import org.apache.spark.sql.catalyst.trees
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 /**
  * An expression that produces zero or more rows given a single input row.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
index 94e1d37..8ee4bbd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
@@ -19,8 +19,8 @@ package org.apache.spark.sql.catalyst.expressions
 
 import java.sql.{Date, Timestamp}
 
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
 
 object Literal {
   def apply(v: Any): Literal = v match {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 a3c300b..3035d93 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
@@ -20,8 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
 import org.apache.spark.sql.catalyst.trees
 import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
 import org.apache.spark.sql.catalyst.errors.TreeNodeException
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.catalyst.util.Metadata
+import org.apache.spark.sql.types._
 
 object NamedExpression {
   private val curId = new java.util.concurrent.atomic.AtomicLong()

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
index cb5ff67..c84cc95 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
 
 import org.apache.spark.sql.catalyst.analysis.UnresolvedException
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.types.BooleanType
+import org.apache.spark.sql.types.BooleanType
 
 object InterpretedPredicate {
   def apply(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) =

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
index 3d4c4a8..3a5bdca 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.expressions
 
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 import org.apache.spark.util.collection.OpenHashSet
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
index f634976..f85ee0a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
@@ -23,7 +23,7 @@ import scala.collection.IndexedSeqOptimized
 
 
 import org.apache.spark.sql.catalyst.analysis.UnresolvedException
-import org.apache.spark.sql.catalyst.types.{BinaryType, BooleanType, DataType, StringType}
+import org.apache.spark.sql.types.{BinaryType, BooleanType, DataType, StringType}
 
 trait StringRegexExpression {
   self: BinaryExpression =>

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index cd31379..17b4f9c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -26,8 +26,8 @@ import org.apache.spark.sql.catalyst.plans.RightOuter
 import org.apache.spark.sql.catalyst.plans.LeftSemi
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
 
 abstract class Optimizer extends RuleExecutor[LogicalPlan]
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
index a38079c..105cdf5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
@@ -27,6 +27,6 @@ package object catalyst {
    * scala.reflect.*.  Note that Scala Reflection API is made thread-safe in 2.11, but not yet for
    * 2.10.* builds.  See SI-6240 for more details.
    */
-  protected[catalyst] object ScalaReflectionLock
+  protected[sql] object ScalaReflectionLock
 
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
index dcbbb62..619f428 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.plans
 
 import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression}
 import org.apache.spark.sql.catalyst.trees.TreeNode
-import org.apache.spark.sql.catalyst.types.{ArrayType, DataType, StructField, StructType}
+import org.apache.spark.sql.types.{ArrayType, DataType, StructField, StructType}
 
 abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] {
   self: PlanType with Product =>

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
index ed578e0..65ae066 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.QueryPlan
 import org.apache.spark.sql.catalyst.trees.TreeNode
-import org.apache.spark.sql.catalyst.types.StructType
+import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.catalyst.trees
 
 /**
@@ -191,14 +191,13 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging {
       case (Nil, _) => expression
       case (requestedField :: rest, StructType(fields)) =>
         val actualField = fields.filter(f => resolver(f.name, requestedField))
-        actualField match {
-          case Seq() =>
-            sys.error(
-              s"No such struct field $requestedField in ${fields.map(_.name).mkString(", ")}")
-          case Seq(singleMatch) =>
-            resolveNesting(rest, GetField(expression, singleMatch.name), resolver)
-          case multipleMatches =>
-            sys.error(s"Ambiguous reference to fields ${multipleMatches.mkString(", ")}")
+        if (actualField.length == 0) {
+          sys.error(
+            s"No such struct field $requestedField in ${fields.map(_.name).mkString(", ")}")
+        } else if (actualField.length == 1) {
+          resolveNesting(rest, GetField(expression, actualField(0).name), resolver)
+        } else {
+          sys.error(s"Ambiguous reference to fields ${actualField.mkString(", ")}")
         }
       case (_, dt) => sys.error(s"Can't access nested field in type $dt")
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
index 0b9f01c..1483bea 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.plans.logical
 
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode {
   def output = projectList.map(_.toAttribute)

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
index ccb0df1..3c3d7a3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.plans.physical
 
 import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.expressions.{Expression, Row, SortOrder}
-import org.apache.spark.sql.catalyst.types.IntegerType
+import org.apache.spark.sql.types.IntegerType
 
 /**
  * Specifies how tuples that share common expressions will be distributed when a query is executed

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
deleted file mode 100644
index 892b7e1..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
+++ /dev/null
@@ -1,650 +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.catalyst.types
-
-import java.sql.{Date, Timestamp}
-
-import scala.math.Numeric.{FloatAsIfIntegral, BigDecimalAsIfIntegral, DoubleAsIfIntegral}
-import scala.reflect.ClassTag
-import scala.reflect.runtime.universe.{TypeTag, runtimeMirror, typeTag}
-import scala.util.parsing.combinator.RegexParsers
-
-import org.json4s._
-import org.json4s.JsonAST.JValue
-import org.json4s.JsonDSL._
-import org.json4s.jackson.JsonMethods._
-
-import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.sql.catalyst.ScalaReflectionLock
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, Row}
-import org.apache.spark.sql.catalyst.types.decimal._
-import org.apache.spark.sql.catalyst.util.Metadata
-import org.apache.spark.util.Utils
-
-object DataType {
-  def fromJson(json: String): DataType = parseDataType(parse(json))
-
-  private object JSortedObject {
-    def unapplySeq(value: JValue): Option[List[(String, JValue)]] = value match {
-      case JObject(seq) => Some(seq.toList.sortBy(_._1))
-      case _ => None
-    }
-  }
-
-  // NOTE: Map fields must be sorted in alphabetical order to keep consistent with the Python side.
-  private def parseDataType(json: JValue): DataType = json match {
-    case JString(name) =>
-      PrimitiveType.nameToType(name)
-
-    case JSortedObject(
-        ("containsNull", JBool(n)),
-        ("elementType", t: JValue),
-        ("type", JString("array"))) =>
-      ArrayType(parseDataType(t), n)
-
-    case JSortedObject(
-        ("keyType", k: JValue),
-        ("type", JString("map")),
-        ("valueContainsNull", JBool(n)),
-        ("valueType", v: JValue)) =>
-      MapType(parseDataType(k), parseDataType(v), n)
-
-    case JSortedObject(
-        ("fields", JArray(fields)),
-        ("type", JString("struct"))) =>
-      StructType(fields.map(parseStructField))
-
-    case JSortedObject(
-        ("class", JString(udtClass)),
-        ("pyClass", _),
-        ("sqlType", _),
-        ("type", JString("udt"))) =>
-      Class.forName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]]
-  }
-
-  private def parseStructField(json: JValue): StructField = json match {
-    case JSortedObject(
-        ("metadata", metadata: JObject),
-        ("name", JString(name)),
-        ("nullable", JBool(nullable)),
-        ("type", dataType: JValue)) =>
-      StructField(name, parseDataType(dataType), nullable, Metadata.fromJObject(metadata))
-    // Support reading schema when 'metadata' is missing.
-    case JSortedObject(
-        ("name", JString(name)),
-        ("nullable", JBool(nullable)),
-        ("type", dataType: JValue)) =>
-      StructField(name, parseDataType(dataType), nullable)
-  }
-
-  @deprecated("Use DataType.fromJson instead", "1.2.0")
-  def fromCaseClassString(string: String): DataType = CaseClassStringParser(string)
-
-  private object CaseClassStringParser extends RegexParsers {
-    protected lazy val primitiveType: Parser[DataType] =
-      ( "StringType" ^^^ StringType
-      | "FloatType" ^^^ FloatType
-      | "IntegerType" ^^^ IntegerType
-      | "ByteType" ^^^ ByteType
-      | "ShortType" ^^^ ShortType
-      | "DoubleType" ^^^ DoubleType
-      | "LongType" ^^^ LongType
-      | "BinaryType" ^^^ BinaryType
-      | "BooleanType" ^^^ BooleanType
-      | "DateType" ^^^ DateType
-      | "DecimalType()" ^^^ DecimalType.Unlimited
-      | fixedDecimalType
-      | "TimestampType" ^^^ TimestampType
-      )
-
-    protected lazy val fixedDecimalType: Parser[DataType] =
-      ("DecimalType(" ~> "[0-9]+".r) ~ ("," ~> "[0-9]+".r <~ ")") ^^ {
-        case precision ~ scale => DecimalType(precision.toInt, scale.toInt)
-      }
-
-    protected lazy val arrayType: Parser[DataType] =
-      "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ {
-        case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull)
-      }
-
-    protected lazy val mapType: Parser[DataType] =
-      "MapType" ~> "(" ~> dataType ~ "," ~ dataType ~ "," ~ boolVal <~ ")" ^^ {
-        case t1 ~ _ ~ t2 ~ _ ~ valueContainsNull => MapType(t1, t2, valueContainsNull)
-      }
-
-    protected lazy val structField: Parser[StructField] =
-      ("StructField(" ~> "[a-zA-Z0-9_]*".r) ~ ("," ~> dataType) ~ ("," ~> boolVal <~ ")") ^^ {
-        case name ~ tpe ~ nullable  =>
-          StructField(name, tpe, nullable = nullable)
-      }
-
-    protected lazy val boolVal: Parser[Boolean] =
-      ( "true" ^^^ true
-      | "false" ^^^ false
-      )
-
-    protected lazy val structType: Parser[DataType] =
-      "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ {
-        case fields => new StructType(fields)
-      }
-
-    protected lazy val dataType: Parser[DataType] =
-      ( arrayType
-      | mapType
-      | structType
-      | primitiveType
-      )
-
-    /**
-     * Parses a string representation of a DataType.
-     *
-     * TODO: Generate parser as pickler...
-     */
-    def apply(asString: String): DataType = parseAll(dataType, asString) match {
-      case Success(result, _) => result
-      case failure: NoSuccess =>
-        throw new IllegalArgumentException(s"Unsupported dataType: $asString, $failure")
-    }
-
-  }
-
-  protected[types] def buildFormattedString(
-      dataType: DataType,
-      prefix: String,
-      builder: StringBuilder): Unit = {
-    dataType match {
-      case array: ArrayType =>
-        array.buildFormattedString(prefix, builder)
-      case struct: StructType =>
-        struct.buildFormattedString(prefix, builder)
-      case map: MapType =>
-        map.buildFormattedString(prefix, builder)
-      case _ =>
-    }
-  }
-
-  /**
-   * Compares two types, ignoring nullability of ArrayType, MapType, StructType.
-   */
-  def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = {
-    (left, right) match {
-      case (ArrayType(leftElementType, _), ArrayType(rightElementType, _)) =>
-        equalsIgnoreNullability(leftElementType, rightElementType)
-      case (MapType(leftKeyType, leftValueType, _), MapType(rightKeyType, rightValueType, _)) =>
-        equalsIgnoreNullability(leftKeyType, rightKeyType) &&
-        equalsIgnoreNullability(leftValueType, rightValueType)
-      case (StructType(leftFields), StructType(rightFields)) =>
-        leftFields.size == rightFields.size &&
-        leftFields.zip(rightFields)
-          .forall{
-            case (left, right) =>
-              left.name == right.name && equalsIgnoreNullability(left.dataType, right.dataType)
-          }
-      case (left, right) => left == right
-    }
-  }
-}
-
-abstract class DataType {
-  /** Matches any expression that evaluates to this DataType */
-  def unapply(a: Expression): Boolean = a match {
-    case e: Expression if e.dataType == this => true
-    case _ => false
-  }
-
-  def isPrimitive: Boolean = false
-
-  def typeName: String = this.getClass.getSimpleName.stripSuffix("$").dropRight(4).toLowerCase
-
-  private[sql] def jsonValue: JValue = typeName
-
-  def json: String = compact(render(jsonValue))
-
-  def prettyJson: String = pretty(render(jsonValue))
-}
-
-case object NullType extends DataType
-
-object NativeType {
-  val all = Seq(
-    IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType)
-
-  def unapply(dt: DataType): Boolean = all.contains(dt)
-
-  val defaultSizeOf: Map[NativeType, Int] = Map(
-    IntegerType -> 4,
-    BooleanType -> 1,
-    LongType -> 8,
-    DoubleType -> 8,
-    FloatType -> 4,
-    ShortType -> 2,
-    ByteType -> 1,
-    StringType -> 4096)
-}
-
-trait PrimitiveType extends DataType {
-  override def isPrimitive = true
-}
-
-object PrimitiveType {
-  private val nonDecimals = Seq(NullType, DateType, TimestampType, BinaryType) ++ NativeType.all
-  private val nonDecimalNameToType = nonDecimals.map(t => t.typeName -> t).toMap
-
-  /** Given the string representation of a type, return its DataType */
-  private[sql] def nameToType(name: String): DataType = {
-    val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\d+)\s*\)""".r
-    name match {
-      case "decimal" => DecimalType.Unlimited
-      case FIXED_DECIMAL(precision, scale) => DecimalType(precision.toInt, scale.toInt)
-      case other => nonDecimalNameToType(other)
-    }
-  }
-}
-
-abstract class NativeType extends DataType {
-  private[sql] type JvmType
-  @transient private[sql] val tag: TypeTag[JvmType]
-  private[sql] val ordering: Ordering[JvmType]
-
-  @transient private[sql] val classTag = ScalaReflectionLock.synchronized {
-    val mirror = runtimeMirror(Utils.getSparkClassLoader)
-    ClassTag[JvmType](mirror.runtimeClass(tag.tpe))
-  }
-}
-
-case object StringType extends NativeType with PrimitiveType {
-  private[sql] type JvmType = String
-  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
-  private[sql] val ordering = implicitly[Ordering[JvmType]]
-}
-
-case object BinaryType extends NativeType with PrimitiveType {
-  private[sql] type JvmType = Array[Byte]
-  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
-  private[sql] val ordering = new Ordering[JvmType] {
-    def compare(x: Array[Byte], y: Array[Byte]): Int = {
-      for (i <- 0 until x.length; if i < y.length) {
-        val res = x(i).compareTo(y(i))
-        if (res != 0) return res
-      }
-      x.length - y.length
-    }
-  }
-}
-
-case object BooleanType extends NativeType with PrimitiveType {
-  private[sql] type JvmType = Boolean
-  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
-  private[sql] val ordering = implicitly[Ordering[JvmType]]
-}
-
-case object TimestampType extends NativeType {
-  private[sql] type JvmType = Timestamp
-
-  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
-
-  private[sql] val ordering = new Ordering[JvmType] {
-    def compare(x: Timestamp, y: Timestamp) = x.compareTo(y)
-  }
-}
-
-case object DateType extends NativeType {
-  private[sql] type JvmType = Date
-
-  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
-
-  private[sql] val ordering = new Ordering[JvmType] {
-    def compare(x: Date, y: Date) = x.compareTo(y)
-  }
-}
-
-abstract class NumericType extends NativeType with PrimitiveType {
-  // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for
-  // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a
-  // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets
-  // desugared by the compiler into an argument to the objects constructor. This means there is no
-  // longer an no argument constructor and thus the JVM cannot serialize the object anymore.
-  private[sql] val numeric: Numeric[JvmType]
-}
-
-object NumericType {
-  def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[NumericType]
-}
-
-/** Matcher for any expressions that evaluate to [[IntegralType]]s */
-object IntegralType {
-  def unapply(a: Expression): Boolean = a match {
-    case e: Expression if e.dataType.isInstanceOf[IntegralType] => true
-    case _ => false
-  }
-}
-
-abstract class IntegralType extends NumericType {
-  private[sql] val integral: Integral[JvmType]
-}
-
-case object LongType extends IntegralType {
-  private[sql] type JvmType = Long
-  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
-  private[sql] val numeric = implicitly[Numeric[Long]]
-  private[sql] val integral = implicitly[Integral[Long]]
-  private[sql] val ordering = implicitly[Ordering[JvmType]]
-}
-
-case object IntegerType extends IntegralType {
-  private[sql] type JvmType = Int
-  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
-  private[sql] val numeric = implicitly[Numeric[Int]]
-  private[sql] val integral = implicitly[Integral[Int]]
-  private[sql] val ordering = implicitly[Ordering[JvmType]]
-}
-
-case object ShortType extends IntegralType {
-  private[sql] type JvmType = Short
-  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
-  private[sql] val numeric = implicitly[Numeric[Short]]
-  private[sql] val integral = implicitly[Integral[Short]]
-  private[sql] val ordering = implicitly[Ordering[JvmType]]
-}
-
-case object ByteType extends IntegralType {
-  private[sql] type JvmType = Byte
-  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
-  private[sql] val numeric = implicitly[Numeric[Byte]]
-  private[sql] val integral = implicitly[Integral[Byte]]
-  private[sql] val ordering = implicitly[Ordering[JvmType]]
-}
-
-/** Matcher for any expressions that evaluate to [[FractionalType]]s */
-object FractionalType {
-  def unapply(a: Expression): Boolean = a match {
-    case e: Expression if e.dataType.isInstanceOf[FractionalType] => true
-    case _ => false
-  }
-}
-
-abstract class FractionalType extends NumericType {
-  private[sql] val fractional: Fractional[JvmType]
-  private[sql] val asIntegral: Integral[JvmType]
-}
-
-/** Precision parameters for a Decimal */
-case class PrecisionInfo(precision: Int, scale: Int)
-
-/** A Decimal that might have fixed precision and scale, or unlimited values for these */
-case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalType {
-  private[sql] type JvmType = Decimal
-  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
-  private[sql] val numeric = Decimal.DecimalIsFractional
-  private[sql] val fractional = Decimal.DecimalIsFractional
-  private[sql] val ordering = Decimal.DecimalIsFractional
-  private[sql] val asIntegral = Decimal.DecimalAsIfIntegral
-
-  override def typeName: String = precisionInfo match {
-    case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)"
-    case None => "decimal"
-  }
-
-  override def toString: String = precisionInfo match {
-    case Some(PrecisionInfo(precision, scale)) => s"DecimalType($precision,$scale)"
-    case None => "DecimalType()"
-  }
-}
-
-/** Extra factory methods and pattern matchers for Decimals */
-object DecimalType {
-  val Unlimited: DecimalType = DecimalType(None)
-
-  object Fixed {
-    def unapply(t: DecimalType): Option[(Int, Int)] =
-      t.precisionInfo.map(p => (p.precision, p.scale))
-  }
-
-  object Expression {
-    def unapply(e: Expression): Option[(Int, Int)] = e.dataType match {
-      case t: DecimalType => t.precisionInfo.map(p => (p.precision, p.scale))
-      case _ => None
-    }
-  }
-
-  def apply(): DecimalType = Unlimited
-
-  def apply(precision: Int, scale: Int): DecimalType =
-    DecimalType(Some(PrecisionInfo(precision, scale)))
-
-  def unapply(t: DataType): Boolean = t.isInstanceOf[DecimalType]
-
-  def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[DecimalType]
-
-  def isFixed(dataType: DataType): Boolean = dataType match {
-    case DecimalType.Fixed(_, _) => true
-    case _ => false
-  }
-}
-
-case object DoubleType extends FractionalType {
-  private[sql] type JvmType = Double
-  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
-  private[sql] val numeric = implicitly[Numeric[Double]]
-  private[sql] val fractional = implicitly[Fractional[Double]]
-  private[sql] val ordering = implicitly[Ordering[JvmType]]
-  private[sql] val asIntegral = DoubleAsIfIntegral
-}
-
-case object FloatType extends FractionalType {
-  private[sql] type JvmType = Float
-  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
-  private[sql] val numeric = implicitly[Numeric[Float]]
-  private[sql] val fractional = implicitly[Fractional[Float]]
-  private[sql] val ordering = implicitly[Ordering[JvmType]]
-  private[sql] val asIntegral = FloatAsIfIntegral
-}
-
-object ArrayType {
-  /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */
-  def apply(elementType: DataType): ArrayType = ArrayType(elementType, true)
-}
-
-/**
- * The data type for collections of multiple values.
- * Internally these are represented as columns that contain a ``scala.collection.Seq``.
- *
- * @param elementType The data type of values.
- * @param containsNull Indicates if values have `null` values
- */
-case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType {
-  private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
-    builder.append(
-      s"$prefix-- element: ${elementType.typeName} (containsNull = $containsNull)\n")
-    DataType.buildFormattedString(elementType, s"$prefix    |", builder)
-  }
-
-  override private[sql] def jsonValue =
-    ("type" -> typeName) ~
-      ("elementType" -> elementType.jsonValue) ~
-      ("containsNull" -> containsNull)
-}
-
-/**
- * A field inside a StructType.
- * @param name The name of this field.
- * @param dataType The data type of this field.
- * @param nullable Indicates if values of this field can be `null` values.
- * @param metadata The metadata of this field. The metadata should be preserved during
- *                 transformation if the content of the column is not modified, e.g, in selection.
- */
-case class StructField(
-    name: String,
-    dataType: DataType,
-    nullable: Boolean = true,
-    metadata: Metadata = Metadata.empty) {
-
-  private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
-    builder.append(s"$prefix-- $name: ${dataType.typeName} (nullable = $nullable)\n")
-    DataType.buildFormattedString(dataType, s"$prefix    |", builder)
-  }
-
-  // override the default toString to be compatible with legacy parquet files.
-  override def toString: String = s"StructField($name,$dataType,$nullable)"
-
-  private[sql] def jsonValue: JValue = {
-    ("name" -> name) ~
-      ("type" -> dataType.jsonValue) ~
-      ("nullable" -> nullable) ~
-      ("metadata" -> metadata.jsonValue)
-  }
-}
-
-object StructType {
-  protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType =
-    StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))
-}
-
-case class StructType(fields: Seq[StructField]) extends DataType {
-
-  /**
-   * Returns all field names in a [[Seq]].
-   */
-  lazy val fieldNames: Seq[String] = fields.map(_.name)
-  private lazy val fieldNamesSet: Set[String] = fieldNames.toSet
-  private lazy val nameToField: Map[String, StructField] = fields.map(f => f.name -> f).toMap
-  /**
-   * Extracts a [[StructField]] of the given name. If the [[StructType]] object does not
-   * have a name matching the given name, `null` will be returned.
-   */
-  def apply(name: String): StructField = {
-    nameToField.getOrElse(name, throw new IllegalArgumentException(s"Field $name does not exist."))
-  }
-
-  /**
-   * Returns a [[StructType]] containing [[StructField]]s of the given names.
-   * Those names which do not have matching fields will be ignored.
-   */
-  def apply(names: Set[String]): StructType = {
-    val nonExistFields = names -- fieldNamesSet
-    if (nonExistFields.nonEmpty) {
-      throw new IllegalArgumentException(
-        s"Field ${nonExistFields.mkString(",")} does not exist.")
-    }
-    // Preserve the original order of fields.
-    StructType(fields.filter(f => names.contains(f.name)))
-  }
-
-  protected[sql] def toAttributes =
-    fields.map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)())
-
-  def treeString: String = {
-    val builder = new StringBuilder
-    builder.append("root\n")
-    val prefix = " |"
-    fields.foreach(field => field.buildFormattedString(prefix, builder))
-
-    builder.toString()
-  }
-
-  def printTreeString(): Unit = println(treeString)
-
-  private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
-    fields.foreach(field => field.buildFormattedString(prefix, builder))
-  }
-
-  override private[sql] def jsonValue =
-    ("type" -> typeName) ~
-      ("fields" -> fields.map(_.jsonValue))
-}
-
-object MapType {
-  /**
-   * Construct a [[MapType]] object with the given key type and value type.
-   * The `valueContainsNull` is true.
-   */
-  def apply(keyType: DataType, valueType: DataType): MapType =
-    MapType(keyType: DataType, valueType: DataType, true)
-}
-
-/**
- * The data type for Maps. Keys in a map are not allowed to have `null` values.
- * @param keyType The data type of map keys.
- * @param valueType The data type of map values.
- * @param valueContainsNull Indicates if map values have `null` values.
- */
-case class MapType(
-    keyType: DataType,
-    valueType: DataType,
-    valueContainsNull: Boolean) extends DataType {
-  private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
-    builder.append(s"$prefix-- key: ${keyType.typeName}\n")
-    builder.append(s"$prefix-- value: ${valueType.typeName} " +
-      s"(valueContainsNull = $valueContainsNull)\n")
-    DataType.buildFormattedString(keyType, s"$prefix    |", builder)
-    DataType.buildFormattedString(valueType, s"$prefix    |", builder)
-  }
-
-  override private[sql] def jsonValue: JValue =
-    ("type" -> typeName) ~
-      ("keyType" -> keyType.jsonValue) ~
-      ("valueType" -> valueType.jsonValue) ~
-      ("valueContainsNull" -> valueContainsNull)
-}
-
-/**
- * ::DeveloperApi::
- * The data type for User Defined Types (UDTs).
- *
- * This interface allows a user to make their own classes more interoperable with SparkSQL;
- * e.g., by creating a [[UserDefinedType]] for a class X, it becomes possible to create
- * a SchemaRDD which has class X in the schema.
- *
- * For SparkSQL to recognize UDTs, the UDT must be annotated with
- * [[org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType]].
- *
- * The conversion via `serialize` occurs when instantiating a `SchemaRDD` from another RDD.
- * The conversion via `deserialize` occurs when reading from a `SchemaRDD`.
- */
-@DeveloperApi
-abstract class UserDefinedType[UserType] extends DataType with Serializable {
-
-  /** Underlying storage type for this UDT */
-  def sqlType: DataType
-
-  /** Paired Python UDT class, if exists. */
-  def pyUDT: String = null
-
-  /**
-   * Convert the user type to a SQL datum
-   *
-   * TODO: Can we make this take obj: UserType?  The issue is in ScalaReflection.convertToCatalyst,
-   *       where we need to convert Any to UserType.
-   */
-  def serialize(obj: Any): Any
-
-  /** Convert a SQL datum to the user type */
-  def deserialize(datum: Any): UserType
-
-  override private[sql] def jsonValue: JValue = {
-    ("type" -> "udt") ~
-      ("class" -> this.getClass.getName) ~
-      ("pyClass" -> pyUDT) ~
-      ("sqlType" -> sqlType.jsonValue)
-  }
-
-  /**
-   * Class object for the UserType
-   */
-  def userClass: java.lang.Class[UserType]
-}


---------------------------------------------------------------------
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-5123][SQL] Reconcile Java/Scala API for data types.

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala
index 158f26e..4186c27 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.api.java
 
 import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf}
-import org.apache.spark.sql.types.util.DataTypeConversions._
+import org.apache.spark.sql.types.DataType
 
 /**
  * A collection of functions that allow Java users to register UDFs.  In order to handle functions
@@ -38,10 +38,9 @@ private[java] trait UDFRegistration {
      println(s"""
          |def registerFunction(
          |    name: String, f: UDF$i[$extTypeArgs, _], @transient dataType: DataType) = {
-         |  val scalaType = asScalaDataType(dataType)
          |  sqlContext.functionRegistry.registerFunction(
          |    name,
-         |    (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), scalaType, e))
+         |    (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), dataType, e))
          |}
        """.stripMargin)
    }
@@ -94,159 +93,159 @@ private[java] trait UDFRegistration {
   */
 
   // scalastyle:off
-  def registerFunction(name: String, f: UDF1[_, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF1[_, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF2[_, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF2[_, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF3[_, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF3[_, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF4[_, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF4[_, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF5[_, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF5[_, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF6[_, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF6[_, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF7[_, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF7[_, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF8[_, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+
+  def registerFunction(
+      name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
 
-  def registerFunction(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = {
-    val scalaType = asScalaDataType(dataType)
+  def registerFunction(
+      name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = {
     sqlContext.functionRegistry.registerFunction(
       name,
-      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e))
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e))
   }
-
   // scalastyle:on
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala
deleted file mode 100644
index a7d0f4f..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala
+++ /dev/null
@@ -1,75 +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.api.java
-
-import org.apache.spark.sql.catalyst.types.{UserDefinedType => ScalaUserDefinedType}
-import org.apache.spark.sql.{DataType => ScalaDataType}
-import org.apache.spark.sql.types.util.DataTypeConversions
-
-/**
- * Scala wrapper for a Java UserDefinedType
- */
-private[sql] class JavaToScalaUDTWrapper[UserType](val javaUDT: UserDefinedType[UserType])
-  extends ScalaUserDefinedType[UserType] with Serializable {
-
-  /** Underlying storage type for this UDT */
-  val sqlType: ScalaDataType = DataTypeConversions.asScalaDataType(javaUDT.sqlType())
-
-  /** Convert the user type to a SQL datum */
-  def serialize(obj: Any): Any = javaUDT.serialize(obj)
-
-  /** Convert a SQL datum to the user type */
-  def deserialize(datum: Any): UserType = javaUDT.deserialize(datum)
-
-  val userClass: java.lang.Class[UserType] = javaUDT.userClass()
-}
-
-/**
- * Java wrapper for a Scala UserDefinedType
- */
-private[sql] class ScalaToJavaUDTWrapper[UserType](val scalaUDT: ScalaUserDefinedType[UserType])
-  extends UserDefinedType[UserType] with Serializable {
-
-  /** Underlying storage type for this UDT */
-  val sqlType: DataType = DataTypeConversions.asJavaDataType(scalaUDT.sqlType)
-
-  /** Convert the user type to a SQL datum */
-  def serialize(obj: Any): java.lang.Object = scalaUDT.serialize(obj).asInstanceOf[java.lang.Object]
-
-  /** Convert a SQL datum to the user type */
-  def deserialize(datum: Any): UserType = scalaUDT.deserialize(datum)
-
-  val userClass: java.lang.Class[UserType] = scalaUDT.userClass
-}
-
-private[sql] object UDTWrappers {
-
-  def wrapAsScala(udtType: UserDefinedType[_]): ScalaUserDefinedType[_] = {
-    udtType match {
-      case t: ScalaToJavaUDTWrapper[_] => t.scalaUDT
-      case _ => new JavaToScalaUDTWrapper(udtType)
-    }
-  }
-
-  def wrapAsJava(udtType: ScalaUserDefinedType[_]): UserDefinedType[_] = {
-    udtType match {
-      case t: JavaToScalaUDTWrapper[_] => t.javaUDT
-      case _ => new ScalaToJavaUDTWrapper(udtType)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
index 538dd5b..91c4c10 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
@@ -17,11 +17,11 @@
 
 package org.apache.spark.sql.columnar
 
-import java.nio.{ByteOrder, ByteBuffer}
+import java.nio.{ByteBuffer, ByteOrder}
 
-import org.apache.spark.sql.catalyst.types.{BinaryType, NativeType, DataType}
 import org.apache.spark.sql.catalyst.expressions.MutableRow
 import org.apache.spark.sql.columnar.compression.CompressibleColumnAccessor
+import org.apache.spark.sql.types.{BinaryType, DataType, NativeType}
 
 /**
  * An `Iterator` like trait used to extract values from columnar byte buffer. When a value is

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
index c68dcee..3a4977b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
@@ -20,9 +20,9 @@ package org.apache.spark.sql.columnar
 import java.nio.{ByteBuffer, ByteOrder}
 
 import org.apache.spark.sql.Row
-import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.columnar.ColumnBuilder._
 import org.apache.spark.sql.columnar.compression.{AllCompressionSchemes, CompressibleColumnBuilder}
+import org.apache.spark.sql.types._
 
 private[sql] trait ColumnBuilder {
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala
index 668efe4..391b3da 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala
@@ -21,7 +21,7 @@ import java.sql.{Date, Timestamp}
 
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute, AttributeReference}
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 private[sql] class ColumnStatisticsSchema(a: Attribute) extends Serializable {
   val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = true)()

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
index ab66c85..fcf2faa 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
@@ -24,8 +24,8 @@ import scala.reflect.runtime.universe.TypeTag
 
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.expressions.MutableRow
-import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.execution.SparkSqlSerializer
+import org.apache.spark.sql.types._
 
 /**
  * An abstract class that represents type of a column. Used to append/extract Java objects into/from

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala
index 27ac5f4..7dff9de 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala
@@ -18,8 +18,8 @@
 package org.apache.spark.sql.columnar.compression
 
 import org.apache.spark.sql.catalyst.expressions.MutableRow
-import org.apache.spark.sql.catalyst.types.NativeType
 import org.apache.spark.sql.columnar.{ColumnAccessor, NativeColumnAccessor}
+import org.apache.spark.sql.types.NativeType
 
 private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAccessor {
   this: NativeColumnAccessor[T] =>

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
index 628d9ce..aead768 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
@@ -21,8 +21,8 @@ import java.nio.{ByteBuffer, ByteOrder}
 
 import org.apache.spark.Logging
 import org.apache.spark.sql.Row
-import org.apache.spark.sql.catalyst.types.NativeType
 import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder}
+import org.apache.spark.sql.types.NativeType
 
 /**
  * A stackable trait that builds optionally compressed byte buffer for a column.  Memory layout of

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala
index acb06cb..879d29b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala
@@ -21,8 +21,8 @@ import java.nio.{ByteBuffer, ByteOrder}
 
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.expressions.MutableRow
-import org.apache.spark.sql.catalyst.types.NativeType
 import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType}
+import org.apache.spark.sql.types.NativeType
 
 private[sql] trait Encoder[T <: NativeType] {
   def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
index 29edcf1..6467324 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
@@ -25,10 +25,11 @@ import scala.reflect.runtime.universe.runtimeMirror
 
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.expressions.{MutableRow, SpecificMutableRow}
-import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.columnar._
+import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
 
+
 private[sql] case object PassThrough extends CompressionScheme {
   override val typeId = 0
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
index 069e950..20b1483 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
@@ -19,11 +19,12 @@ package org.apache.spark.sql.execution
 
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{StructType, Row, SQLContext}
+import org.apache.spark.sql.{Row, SQLContext}
 import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow}
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics}
+import org.apache.spark.sql.types.StructType
 
 /**
  * :: DeveloperApi ::

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala
index 7c3bf94..4abe26f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala
@@ -21,7 +21,7 @@ import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.sql.catalyst.trees._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 case class AggregateEvaluation(
     schema: Seq[Attribute],

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
index 84d96e6..1311460 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
@@ -29,7 +29,7 @@ import com.twitter.chill.{AllScalaRegistrar, ResourcePool}
 import org.apache.spark.{SparkEnv, SparkConf}
 import org.apache.spark.serializer.{SerializerInstance, KryoSerializer}
 import org.apache.spark.sql.catalyst.expressions.GenericRow
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.apache.spark.sql.types.decimal.Decimal
 import org.apache.spark.util.collection.OpenHashSet
 import org.apache.spark.util.MutablePair
 import org.apache.spark.util.Utils

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 0652d2f..0cc9d04 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -17,16 +17,16 @@
 
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.sources.{CreateTempTableUsing, CreateTableUsing}
 import org.apache.spark.sql.{SQLContext, Strategy, execution}
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.planning._
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan}
+import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation}
 import org.apache.spark.sql.parquet._
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.sources.{CreateTempTableUsing, CreateTableUsing}
 
 
 private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 61be5ed..46245cd 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
@@ -24,7 +24,7 @@ import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.SparkContext._
 import org.apache.spark.sql.{SchemaRDD, Row}
 import org.apache.spark.sql.catalyst.trees.TreeNodeRef
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 /**
  * :: DeveloperApi ::

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
index 5a41399..741ccb8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
@@ -19,8 +19,6 @@ package org.apache.spark.sql.execution
 
 import java.util.{List => JList, Map => JMap}
 
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
-
 import scala.collection.JavaConversions._
 import scala.collection.JavaConverters._
 
@@ -33,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 import org.apache.spark.{Accumulator, Logging => SparkLogging}
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala
index f5c0222..1af96c2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala
@@ -18,8 +18,9 @@
 package org.apache.spark.sql.json
 
 import org.apache.spark.sql.SQLContext
-import org.apache.spark.sql.catalyst.types.StructType
 import org.apache.spark.sql.sources._
+import org.apache.spark.sql.types.StructType
+
 
 private[sql] class DefaultSource extends RelationProvider with SchemaRelationProvider {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
index 00449c2..c92ec54 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
@@ -17,9 +17,6 @@
 
 package org.apache.spark.sql.json
 
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
-import org.apache.spark.sql.types.util.DataTypeConversions
-
 import java.io.StringWriter
 
 import scala.collection.Map
@@ -34,8 +31,9 @@ import com.fasterxml.jackson.databind.ObjectMapper
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.catalyst.ScalaReflection
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
 import org.apache.spark.Logging
 
 private[sql] object JsonRDD extends Logging {
@@ -246,7 +244,7 @@ private[sql] object JsonRDD extends Logging {
         // The value associated with the key is an array.
         // Handle inner structs of an array.
         def buildKeyPathForInnerStructs(v: Any, t: DataType): Seq[(String, DataType)] = t match {
-          case ArrayType(StructType(Nil), containsNull) => {
+          case ArrayType(e: StructType, containsNull) => {
             // The elements of this arrays are structs.
             v.asInstanceOf[Seq[Map[String, Any]]].flatMap(Option(_)).flatMap {
               element => allKeysWithValueTypes(element)

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 1fd8e62..b75266d 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
@@ -117,357 +117,8 @@ package object sql {
   val Row = catalyst.expressions.Row
 
   /**
-   * :: DeveloperApi ::
-   *
-   * The base type of all Spark SQL data types.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  type DataType = catalyst.types.DataType
-
-  @DeveloperApi
-  val DataType = catalyst.types.DataType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `String` values
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val StringType = catalyst.types.StringType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `Array[Byte]` values.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val BinaryType = catalyst.types.BinaryType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `Boolean` values.
-   *
-   *@group dataType
-   */
-  @DeveloperApi
-  val BooleanType = catalyst.types.BooleanType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `java.sql.Timestamp` values.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val TimestampType = catalyst.types.TimestampType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `java.sql.Date` values.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val DateType = catalyst.types.DateType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `scala.math.BigDecimal` values.
-   *
-   * TODO(matei): explain precision and scale
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  type DecimalType = catalyst.types.DecimalType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `scala.math.BigDecimal` values.
-   *
-   * TODO(matei): explain precision and scale
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val DecimalType = catalyst.types.DecimalType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `Double` values.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val DoubleType = catalyst.types.DoubleType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `Float` values.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val FloatType = catalyst.types.FloatType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `Byte` values.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val ByteType = catalyst.types.ByteType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `Int` values.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val IntegerType = catalyst.types.IntegerType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `Long` values.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val LongType = catalyst.types.LongType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `Short` values.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val ShortType = catalyst.types.ShortType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `NULL` values.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val NullType = catalyst.types.NullType
-  
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type for collections of multiple values.
-   * Internally these are represented as columns that contain a ``scala.collection.Seq``.
-   *
-   * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and
-   * `containsNull: Boolean`. The field of `elementType` is used to specify the type of
-   * array elements. The field of `containsNull` is used to specify if the array has `null` values.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  type ArrayType = catalyst.types.ArrayType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * An [[ArrayType]] object can be constructed with two ways,
-   * {{{
-   * ArrayType(elementType: DataType, containsNull: Boolean)
-   * }}} and
-   * {{{
-   * ArrayType(elementType: DataType)
-   * }}}
-   * For `ArrayType(elementType)`, the field of `containsNull` is set to `false`.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val ArrayType = catalyst.types.ArrayType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing `Map`s. A [[MapType]] object comprises three fields,
-   * `keyType: [[DataType]]`, `valueType: [[DataType]]` and `valueContainsNull: Boolean`.
-   * The field of `keyType` is used to specify the type of keys in the map.
-   * The field of `valueType` is used to specify the type of values in the map.
-   * The field of `valueContainsNull` is used to specify if values of this map has `null` values.
-   * For values of a MapType column, keys are not allowed to have `null` values.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  type MapType = catalyst.types.MapType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * A [[MapType]] object can be constructed with two ways,
-   * {{{
-   * MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean)
-   * }}} and
-   * {{{
-   * MapType(keyType: DataType, valueType: DataType)
-   * }}}
-   * For `MapType(keyType: DataType, valueType: DataType)`,
-   * the field of `valueContainsNull` is set to `true`.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val MapType = catalyst.types.MapType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * The data type representing [[Row]]s.
-   * A [[StructType]] object comprises a [[Seq]] of [[StructField]]s.
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  type StructType = catalyst.types.StructType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * A [[StructType]] object can be constructed by
-   * {{{
-   * StructType(fields: Seq[StructField])
-   * }}}
-   * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names.
-   * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned.
-   * If a provided name does not have a matching field, it will be ignored. For the case
-   * of extracting a single StructField, a `null` will be returned.
-   * Example:
-   * {{{
-   * import org.apache.spark.sql._
-   *
-   * val struct =
-   *   StructType(
-   *     StructField("a", IntegerType, true) ::
-   *     StructField("b", LongType, false) ::
-   *     StructField("c", BooleanType, false) :: Nil)
-   *
-   * // Extract a single StructField.
-   * val singleField = struct("b")
-   * // singleField: StructField = StructField(b,LongType,false)
-   *
-   * // This struct does not have a field called "d". null will be returned.
-   * val nonExisting = struct("d")
-   * // nonExisting: StructField = null
-   *
-   * // Extract multiple StructFields. Field names are provided in a set.
-   * // A StructType object will be returned.
-   * val twoFields = struct(Set("b", "c"))
-   * // twoFields: StructType =
-   * //   StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false)))
-   *
-   * // Those names do not have matching fields will be ignored.
-   * // For the case shown below, "d" will be ignored and
-   * // it is treated as struct(Set("b", "c")).
-   * val ignoreNonExisting = struct(Set("b", "c", "d"))
-   * // ignoreNonExisting: StructType =
-   * //   StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false)))
-   * }}}
-   *
-   * A [[Row]] object is used as a value of the StructType.
-   * Example:
-   * {{{
-   * import org.apache.spark.sql._
-   *
-   * val innerStruct =
-   *   StructType(
-   *     StructField("f1", IntegerType, true) ::
-   *     StructField("f2", LongType, false) ::
-   *     StructField("f3", BooleanType, false) :: Nil)
-   *
-   * val struct = StructType(
-   *   StructField("a", innerStruct, true) :: Nil)
-   *
-   * // Create a Row with the schema defined by struct
-   * val row = Row(Row(1, 2, true))
-   * // row: Row = [[1,2,true]]
-   * }}}
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val StructType = catalyst.types.StructType
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * A [[StructField]] object represents a field in a [[StructType]] object.
-   * A [[StructField]] object comprises three fields, `name: [[String]]`, `dataType: [[DataType]]`,
-   * and `nullable: Boolean`. The field of `name` is the name of a `StructField`. The field of
-   * `dataType` specifies the data type of a `StructField`.
-   * The field of `nullable` specifies if values of a `StructField` can contain `null` values.
-   *
-   * @group field
-   */
-  @DeveloperApi
-  type StructField = catalyst.types.StructField
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * A [[StructField]] object can be constructed by
-   * {{{
-   * StructField(name: String, dataType: DataType, nullable: Boolean)
-   * }}}
-   *
-   * @group dataType
-   */
-  @DeveloperApi
-  val StructField = catalyst.types.StructField
-
-  /**
    * Converts a logical plan into zero or more SparkPlans.
    */
   @DeveloperApi
   type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan]
-
-  /**
-   * :: DeveloperApi ::
-   *
-   * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean,
-   * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and
-   * Array[Metadata]. JSON is used for serialization.
-   *
-   * The default constructor is private. User should use either [[MetadataBuilder]] or
-   * [[Metadata$#fromJson]] to create Metadata instances.
-   *
-   * @param map an immutable map that stores the data
-   */
-  @DeveloperApi
-  type Metadata = catalyst.util.Metadata
-
-  /**
-   * :: DeveloperApi ::
-   * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former.
-   */
-  @DeveloperApi
-  type MetadataBuilder = catalyst.util.MetadataBuilder
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala
index 1bbb66a..7f437c4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala
@@ -17,16 +17,15 @@
 
 package org.apache.spark.sql.parquet
 
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
-
 import scala.collection.mutable.{Buffer, ArrayBuffer, HashMap}
 
 import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter}
 import parquet.schema.MessageType
 
-import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.parquet.CatalystConverter.FieldType
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
 
 /**
  * Collection of converters of Parquet types (group and primitive types) that
@@ -91,8 +90,8 @@ private[sql] object CatalystConverter {
       case ArrayType(elementType: DataType, true) => {
         new CatalystArrayContainsNullConverter(elementType, fieldIndex, parent)
       }
-      case StructType(fields: Seq[StructField]) => {
-        new CatalystStructConverter(fields.toArray, fieldIndex, parent)
+      case StructType(fields: Array[StructField]) => {
+        new CatalystStructConverter(fields, fieldIndex, parent)
       }
       case MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean) => {
         new CatalystMapConverter(
@@ -436,7 +435,7 @@ private[parquet] object CatalystArrayConverter {
  * A `parquet.io.api.GroupConverter` that converts a single-element groups that
  * match the characteristics of an array (see
  * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an
- * [[org.apache.spark.sql.catalyst.types.ArrayType]].
+ * [[org.apache.spark.sql.types.ArrayType]].
  *
  * @param elementType The type of the array elements (complex or primitive)
  * @param index The position of this (array) field inside its parent converter
@@ -500,7 +499,7 @@ private[parquet] class CatalystArrayConverter(
  * A `parquet.io.api.GroupConverter` that converts a single-element groups that
  * match the characteristics of an array (see
  * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an
- * [[org.apache.spark.sql.catalyst.types.ArrayType]].
+ * [[org.apache.spark.sql.types.ArrayType]].
  *
  * @param elementType The type of the array elements (native)
  * @param index The position of this (array) field inside its parent converter
@@ -621,7 +620,7 @@ private[parquet] class CatalystNativeArrayConverter(
  * A `parquet.io.api.GroupConverter` that converts a single-element groups that
  * match the characteristics of an array contains null (see
  * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an
- * [[org.apache.spark.sql.catalyst.types.ArrayType]].
+ * [[org.apache.spark.sql.types.ArrayType]].
  *
  * @param elementType The type of the array elements (complex or primitive)
  * @param index The position of this (array) field inside its parent converter
@@ -727,7 +726,7 @@ private[parquet] class CatalystStructConverter(
  * A `parquet.io.api.GroupConverter` that converts two-element groups that
  * match the characteristics of a map (see
  * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an
- * [[org.apache.spark.sql.catalyst.types.MapType]].
+ * [[org.apache.spark.sql.types.MapType]].
  *
  * @param schema
  * @param index

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala
index 56e7d11..f083508 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala
@@ -29,7 +29,7 @@ import parquet.io.api.Binary
 
 import org.apache.spark.SparkEnv
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 private[sql] object ParquetFilters {
   val PARQUET_FILTER_DATA = "org.apache.spark.sql.parquet.row.filter"

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
index 9049eb5..af7248f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
@@ -29,8 +29,8 @@ import parquet.schema.MessageType
 
 import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.expressions.{Attribute, Row}
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.decimal.Decimal
 
 /**
  * A `parquet.io.api.RecordMaterializer` for Rows.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala
index 9744787..6d8c682 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala
@@ -36,7 +36,7 @@ import parquet.schema.Type.Repetition
 
 import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute}
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 // Implicits
 import scala.collection.JavaConversions._
@@ -80,7 +80,7 @@ private[parquet] object ParquetTypesConverter extends Logging {
 
   /**
    * Converts a given Parquet `Type` into the corresponding
-   * [[org.apache.spark.sql.catalyst.types.DataType]].
+   * [[org.apache.spark.sql.types.DataType]].
    *
    * We apply the following conversion rules:
    * <ul>
@@ -191,7 +191,7 @@ private[parquet] object ParquetTypesConverter extends Logging {
   }
 
   /**
-   * For a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] return
+   * For a given Catalyst [[org.apache.spark.sql.types.DataType]] return
    * the name of the corresponding Parquet primitive type or None if the given type
    * is not primitive.
    *
@@ -231,21 +231,21 @@ private[parquet] object ParquetTypesConverter extends Logging {
   }
 
   /**
-   * Converts a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] into
+   * Converts a given Catalyst [[org.apache.spark.sql.types.DataType]] into
    * the corresponding Parquet `Type`.
    *
    * The conversion follows the rules below:
    * <ul>
    *   <li> Primitive types are converted into Parquet's primitive types.</li>
-   *   <li> [[org.apache.spark.sql.catalyst.types.StructType]]s are converted
+   *   <li> [[org.apache.spark.sql.types.StructType]]s are converted
    *        into Parquet's `GroupType` with the corresponding field types.</li>
-   *   <li> [[org.apache.spark.sql.catalyst.types.ArrayType]]s are converted
+   *   <li> [[org.apache.spark.sql.types.ArrayType]]s are converted
    *        into a 2-level nested group, where the outer group has the inner
    *        group as sole field. The inner group has name `values` and
    *        repetition level `REPEATED` and has the element type of
    *        the array as schema. We use Parquet's `ConversionPatterns` for this
    *        purpose.</li>
-   *   <li> [[org.apache.spark.sql.catalyst.types.MapType]]s are converted
+   *   <li> [[org.apache.spark.sql.types.MapType]]s are converted
    *        into a nested (2-level) Parquet `GroupType` with two fields: a key
    *        type and a value type. The nested group has repetition level
    *        `REPEATED` and name `map`. We use Parquet's `ConversionPatterns`
@@ -319,7 +319,7 @@ private[parquet] object ParquetTypesConverter extends Logging {
           val fields = structFields.map {
             field => fromDataType(field.dataType, field.name, field.nullable, inArray = false)
           }
-          new ParquetGroupType(repetition, name, fields)
+          new ParquetGroupType(repetition, name, fields.toSeq)
         }
         case MapType(keyType, valueType, valueContainsNull) => {
           val parquetKeyType =

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
index 55a2728..1b50afb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
@@ -18,11 +18,12 @@ package org.apache.spark.sql.parquet
 
 import java.util.{List => JList}
 
+import scala.collection.JavaConversions._
+
 import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
 import org.apache.hadoop.conf.{Configurable, Configuration}
 import org.apache.hadoop.io.Writable
 import org.apache.hadoop.mapreduce.{JobContext, InputSplit, Job}
-import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
 
 import parquet.hadoop.ParquetInputFormat
 import parquet.hadoop.util.ContextUtil
@@ -30,13 +31,11 @@ import parquet.hadoop.util.ContextUtil
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.{Partition => SparkPartition, Logging}
 import org.apache.spark.rdd.{NewHadoopPartition, RDD}
-
 import org.apache.spark.sql.{SQLConf, Row, SQLContext}
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.types.{StringType, IntegerType, StructField, StructType}
 import org.apache.spark.sql.sources._
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
 
-import scala.collection.JavaConversions._
 
 /**
  * Allows creation of parquet based tables using the syntax

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala
index 4d87f68..12b59ba 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala
@@ -17,7 +17,7 @@
 package org.apache.spark.sql.sources
 
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
-import org.apache.spark.sql.catalyst.expressions.AttributeMap
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeMap}
 import org.apache.spark.sql.catalyst.plans.logical.{Statistics, LeafNode, LogicalPlan}
 
 /**
@@ -27,7 +27,7 @@ private[sql] case class LogicalRelation(relation: BaseRelation)
   extends LeafNode
   with MultiInstanceRelation {
 
-  override val output = relation.schema.toAttributes
+  override val output: Seq[AttributeReference] = relation.schema.toAttributes
 
   // Logical Relations are distinct if they have different output for the sake of transformations.
   override def equals(other: Any) = other match {

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 f8741e0..4cc9641 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
@@ -23,11 +23,11 @@ import scala.util.parsing.combinator.PackratParsers
 
 import org.apache.spark.Logging
 import org.apache.spark.sql.SQLContext
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.execution.RunnableCommand
-import org.apache.spark.util.Utils
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.SqlLexical
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
 
 /**
  * A parser for foreign DDL commands.
@@ -162,10 +162,10 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi
 
   protected lazy val structType: Parser[DataType] =
     (STRUCT ~> "<" ~> repsep(structField, ",") <~ ">" ^^ {
-    case fields => new StructType(fields)
+    case fields => StructType(fields)
     }) |
     (STRUCT ~> "<>" ^^ {
-      case fields => new StructType(Nil)
+      case fields => StructType(Nil)
     })
 
   private[sql] lazy val dataType: Parser[DataType] =

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
index 7f5564b..cd82cc6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
@@ -18,8 +18,9 @@ package org.apache.spark.sql.sources
 
 import org.apache.spark.annotation.{Experimental, DeveloperApi}
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{Row, SQLContext, StructType}
+import org.apache.spark.sql.{Row, SQLContext}
 import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute}
+import org.apache.spark.sql.types.StructType
 
 /**
  * ::DeveloperApi::

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala
index b9569e9..006b16f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala
@@ -20,9 +20,7 @@ package org.apache.spark.sql.test
 import java.util
 
 import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 /**
  * An example class to demonstrate UDT in Scala, Java, and Python.

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala
deleted file mode 100644
index d4ef517..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala
+++ /dev/null
@@ -1,175 +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.types.util
-
-import java.text.SimpleDateFormat
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql._
-import org.apache.spark.sql.api.java.{DataType => JDataType, StructField => JStructField,
-  MetadataBuilder => JMetaDataBuilder, UDTWrappers}
-import org.apache.spark.sql.api.java.{DecimalType => JDecimalType}
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
-import org.apache.spark.sql.catalyst.ScalaReflection
-import org.apache.spark.sql.catalyst.types.UserDefinedType
-
-protected[sql] object DataTypeConversions {
-
-  /**
-   * Returns the equivalent StructField in Scala for the given StructField in Java.
-   */
-  def asJavaStructField(scalaStructField: StructField): JStructField = {
-    JDataType.createStructField(
-      scalaStructField.name,
-      asJavaDataType(scalaStructField.dataType),
-      scalaStructField.nullable,
-      (new JMetaDataBuilder).withMetadata(scalaStructField.metadata).build())
-  }
-
-  /**
-   * Returns the equivalent DataType in Java for the given DataType in Scala.
-   */
-  def asJavaDataType(scalaDataType: DataType): JDataType = scalaDataType match {
-    case udtType: UserDefinedType[_] =>
-      UDTWrappers.wrapAsJava(udtType)
-
-    case StringType => JDataType.StringType
-    case BinaryType => JDataType.BinaryType
-    case BooleanType => JDataType.BooleanType
-    case DateType => JDataType.DateType
-    case TimestampType => JDataType.TimestampType
-    case DecimalType.Fixed(precision, scale) => new JDecimalType(precision, scale)
-    case DecimalType.Unlimited => new JDecimalType()
-    case DoubleType => JDataType.DoubleType
-    case FloatType => JDataType.FloatType
-    case ByteType => JDataType.ByteType
-    case IntegerType => JDataType.IntegerType
-    case LongType => JDataType.LongType
-    case ShortType => JDataType.ShortType
-    case NullType => JDataType.NullType
-
-    case arrayType: ArrayType => JDataType.createArrayType(
-        asJavaDataType(arrayType.elementType), arrayType.containsNull)
-    case mapType: MapType => JDataType.createMapType(
-        asJavaDataType(mapType.keyType),
-        asJavaDataType(mapType.valueType),
-        mapType.valueContainsNull)
-    case structType: StructType => JDataType.createStructType(
-        structType.fields.map(asJavaStructField).asJava)
-  }
-
-  /**
-   * Returns the equivalent StructField in Scala for the given StructField in Java.
-   */
-  def asScalaStructField(javaStructField: JStructField): StructField = {
-    StructField(
-      javaStructField.getName,
-      asScalaDataType(javaStructField.getDataType),
-      javaStructField.isNullable,
-      javaStructField.getMetadata)
-  }
-
-  /**
-   * Returns the equivalent DataType in Scala for the given DataType in Java.
-   */
-  def asScalaDataType(javaDataType: JDataType): DataType = javaDataType match {
-    case udtType: org.apache.spark.sql.api.java.UserDefinedType[_] =>
-      UDTWrappers.wrapAsScala(udtType)
-
-    case stringType: org.apache.spark.sql.api.java.StringType =>
-      StringType
-    case binaryType: org.apache.spark.sql.api.java.BinaryType =>
-      BinaryType
-    case booleanType: org.apache.spark.sql.api.java.BooleanType =>
-      BooleanType
-    case dateType: org.apache.spark.sql.api.java.DateType =>
-      DateType
-    case timestampType: org.apache.spark.sql.api.java.TimestampType =>
-      TimestampType
-    case decimalType: org.apache.spark.sql.api.java.DecimalType =>
-      if (decimalType.isFixed) {
-        DecimalType(decimalType.getPrecision, decimalType.getScale)
-      } else {
-        DecimalType.Unlimited
-      }
-    case doubleType: org.apache.spark.sql.api.java.DoubleType =>
-      DoubleType
-    case floatType: org.apache.spark.sql.api.java.FloatType =>
-      FloatType
-    case byteType: org.apache.spark.sql.api.java.ByteType =>
-      ByteType
-    case integerType: org.apache.spark.sql.api.java.IntegerType =>
-      IntegerType
-    case longType: org.apache.spark.sql.api.java.LongType =>
-      LongType
-    case shortType: org.apache.spark.sql.api.java.ShortType =>
-      ShortType
-
-    case arrayType: org.apache.spark.sql.api.java.ArrayType =>
-      ArrayType(asScalaDataType(arrayType.getElementType), arrayType.isContainsNull)
-    case mapType: org.apache.spark.sql.api.java.MapType =>
-      MapType(
-        asScalaDataType(mapType.getKeyType),
-        asScalaDataType(mapType.getValueType),
-        mapType.isValueContainsNull)
-    case structType: org.apache.spark.sql.api.java.StructType =>
-      StructType(structType.getFields.map(asScalaStructField))
-  }
-
-  def stringToTime(s: String): java.util.Date = {
-    if (!s.contains('T')) {
-      // JDBC escape string
-      if (s.contains(' ')) {
-        java.sql.Timestamp.valueOf(s)
-      } else {
-        java.sql.Date.valueOf(s)
-      }
-    } else if (s.endsWith("Z")) {
-      // this is zero timezone of ISO8601
-      stringToTime(s.substring(0, s.length - 1) + "GMT-00:00")
-    } else if (s.indexOf("GMT") == -1) {
-      // timezone with ISO8601
-      val inset = "+00.00".length
-      val s0 = s.substring(0, s.length - inset)
-      val s1 = s.substring(s.length - inset, s.length)
-      if (s0.substring(s0.lastIndexOf(':')).contains('.')) {
-        stringToTime(s0 + "GMT" + s1)
-      } else {
-        stringToTime(s0 + ".0GMT" + s1)
-      }
-    } else {
-      // ISO8601 with GMT insert
-      val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd'T'HH:mm:ss.SSSz" )
-      ISO8601GMT.parse(s)
-    }
-  }
-
-  /** Converts Java objects to catalyst rows / types */
-  def convertJavaToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match {
-    case (obj, udt: UserDefinedType[_]) => ScalaReflection.convertToCatalyst(obj, udt) // Scala type
-    case (d: java.math.BigDecimal, _) => Decimal(BigDecimal(d))
-    case (other, _) => other
-  }
-
-  /** Converts Java objects to catalyst rows / types */
-  def convertCatalystToJava(a: Any): Any = a match {
-    case d: scala.math.BigDecimal => d.underlying()
-    case other => other
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 a9a1128..88017eb 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
@@ -19,15 +19,12 @@ package org.apache.spark.sql.api.java;
 
 import java.io.Serializable;
 
-import org.apache.spark.sql.api.java.UDF1;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.junit.runners.Suite;
-import org.junit.runner.RunWith;
 
 import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.types.DataTypes;
 
 // The test suite itself is Serializable so that anonymous Function implementations can be
 // serialized, as an alternative to converting these anonymous classes to static inner classes;
@@ -60,7 +57,7 @@ public class JavaAPISuite implements Serializable {
       public Integer call(String str) throws Exception {
         return str.length();
       }
-    }, DataType.IntegerType);
+    }, DataTypes.IntegerType);
 
     // TODO: Why do we need this cast?
     Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test')").first();
@@ -81,7 +78,7 @@ public class JavaAPISuite implements Serializable {
       public Integer call(String str1, String str2) throws Exception {
         return str1.length() + str2.length();
       }
-    }, DataType.IntegerType);
+    }, DataTypes.IntegerType);
 
     // TODO: Why do we need this cast?
     Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test', 'test2')").first();

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/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 a04b806..de586ba 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
@@ -31,6 +31,7 @@ import org.junit.Test;
 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.types.*;
 
 // The test suite itself is Serializable so that anonymous Function implementations can be
 // serialized, as an alternative to converting these anonymous classes to static inner classes;
@@ -93,9 +94,9 @@ public class JavaApplySchemaSuite implements Serializable {
       });
 
     List<StructField> fields = new ArrayList<StructField>(2);
-    fields.add(DataType.createStructField("name", DataType.StringType, false));
-    fields.add(DataType.createStructField("age", DataType.IntegerType, false));
-    StructType schema = DataType.createStructType(fields);
+    fields.add(DataTypes.createStructField("name", DataTypes.StringType, false));
+    fields.add(DataTypes.createStructField("age", DataTypes.IntegerType, false));
+    StructType schema = DataTypes.createStructType(fields);
 
     JavaSchemaRDD schemaRDD = javaSqlCtx.applySchema(rowRDD, schema);
     schemaRDD.registerTempTable("people");
@@ -118,14 +119,14 @@ public class JavaApplySchemaSuite implements Serializable {
         "\"bigInteger\":92233720368547758069, \"double\":1.7976931348623157E305, " +
         "\"boolean\":false, \"null\":null}"));
     List<StructField> fields = new ArrayList<StructField>(7);
-    fields.add(DataType.createStructField("bigInteger", new DecimalType(), true));
-    fields.add(DataType.createStructField("boolean", DataType.BooleanType, true));
-    fields.add(DataType.createStructField("double", DataType.DoubleType, true));
-    fields.add(DataType.createStructField("integer", DataType.IntegerType, true));
-    fields.add(DataType.createStructField("long", DataType.LongType, true));
-    fields.add(DataType.createStructField("null", DataType.StringType, true));
-    fields.add(DataType.createStructField("string", DataType.StringType, true));
-    StructType expectedSchema = DataType.createStructType(fields);
+    fields.add(DataTypes.createStructField("bigInteger", DataTypes.createDecimalType(), true));
+    fields.add(DataTypes.createStructField("boolean", DataTypes.BooleanType, true));
+    fields.add(DataTypes.createStructField("double", DataTypes.DoubleType, true));
+    fields.add(DataTypes.createStructField("integer", DataTypes.IntegerType, true));
+    fields.add(DataTypes.createStructField("long", DataTypes.LongType, true));
+    fields.add(DataTypes.createStructField("null", DataTypes.StringType, true));
+    fields.add(DataTypes.createStructField("string", DataTypes.StringType, true));
+    StructType expectedSchema = DataTypes.createStructType(fields);
     List<Row> expectedResult = new ArrayList<Row>(2);
     expectedResult.add(
       Row.create(

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java
----------------------------------------------------------------------
diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java
deleted file mode 100644
index 8396a29..0000000
--- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java
+++ /dev/null
@@ -1,150 +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.api.java;
-
-import java.util.List;
-import java.util.ArrayList;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.spark.sql.types.util.DataTypeConversions;
-
-public class JavaSideDataTypeConversionSuite {
-  public void checkDataType(DataType javaDataType) {
-    org.apache.spark.sql.catalyst.types.DataType scalaDataType =
-      DataTypeConversions.asScalaDataType(javaDataType);
-    DataType actual = DataTypeConversions.asJavaDataType(scalaDataType);
-    Assert.assertEquals(javaDataType, actual);
-  }
-
-  @Test
-  public void createDataTypes() {
-    // Simple DataTypes.
-    checkDataType(DataType.StringType);
-    checkDataType(DataType.BinaryType);
-    checkDataType(DataType.BooleanType);
-    checkDataType(DataType.DateType);
-    checkDataType(DataType.TimestampType);
-    checkDataType(new DecimalType());
-    checkDataType(new DecimalType(10, 4));
-    checkDataType(DataType.DoubleType);
-    checkDataType(DataType.FloatType);
-    checkDataType(DataType.ByteType);
-    checkDataType(DataType.IntegerType);
-    checkDataType(DataType.LongType);
-    checkDataType(DataType.ShortType);
-
-    // Simple ArrayType.
-    DataType simpleJavaArrayType = DataType.createArrayType(DataType.StringType, true);
-    checkDataType(simpleJavaArrayType);
-
-    // Simple MapType.
-    DataType simpleJavaMapType = DataType.createMapType(DataType.StringType, DataType.LongType);
-    checkDataType(simpleJavaMapType);
-
-    // Simple StructType.
-    List<StructField> simpleFields = new ArrayList<StructField>();
-    simpleFields.add(DataType.createStructField("a", new DecimalType(), false));
-    simpleFields.add(DataType.createStructField("b", DataType.BooleanType, true));
-    simpleFields.add(DataType.createStructField("c", DataType.LongType, true));
-    simpleFields.add(DataType.createStructField("d", DataType.BinaryType, false));
-    DataType simpleJavaStructType = DataType.createStructType(simpleFields);
-    checkDataType(simpleJavaStructType);
-
-    // Complex StructType.
-    List<StructField> complexFields = new ArrayList<StructField>();
-    complexFields.add(DataType.createStructField("simpleArray", simpleJavaArrayType, true));
-    complexFields.add(DataType.createStructField("simpleMap", simpleJavaMapType, true));
-    complexFields.add(DataType.createStructField("simpleStruct", simpleJavaStructType, true));
-    complexFields.add(DataType.createStructField("boolean", DataType.BooleanType, false));
-    DataType complexJavaStructType = DataType.createStructType(complexFields);
-    checkDataType(complexJavaStructType);
-
-    // Complex ArrayType.
-    DataType complexJavaArrayType = DataType.createArrayType(complexJavaStructType, true);
-    checkDataType(complexJavaArrayType);
-
-    // Complex MapType.
-    DataType complexJavaMapType =
-      DataType.createMapType(complexJavaStructType, complexJavaArrayType, false);
-    checkDataType(complexJavaMapType);
-  }
-
-  @Test
-  public void illegalArgument() {
-    // ArrayType
-    try {
-      DataType.createArrayType(null, true);
-      Assert.fail();
-    } catch (IllegalArgumentException expectedException) {
-    }
-
-    // MapType
-    try {
-      DataType.createMapType(null, DataType.StringType);
-      Assert.fail();
-    } catch (IllegalArgumentException expectedException) {
-    }
-    try {
-      DataType.createMapType(DataType.StringType, null);
-      Assert.fail();
-    } catch (IllegalArgumentException expectedException) {
-    }
-    try {
-      DataType.createMapType(null, null);
-      Assert.fail();
-    } catch (IllegalArgumentException expectedException) {
-    }
-
-    // StructField
-    try {
-      DataType.createStructField(null, DataType.StringType, true);
-    } catch (IllegalArgumentException expectedException) {
-    }
-    try {
-      DataType.createStructField("name", null, true);
-    } catch (IllegalArgumentException expectedException) {
-    }
-    try {
-      DataType.createStructField(null, null, true);
-    } catch (IllegalArgumentException expectedException) {
-    }
-
-    // StructType
-    try {
-      List<StructField> simpleFields = new ArrayList<StructField>();
-      simpleFields.add(DataType.createStructField("a", new DecimalType(), false));
-      simpleFields.add(DataType.createStructField("b", DataType.BooleanType, true));
-      simpleFields.add(DataType.createStructField("c", DataType.LongType, true));
-      simpleFields.add(null);
-      DataType.createStructType(simpleFields);
-      Assert.fail();
-    } catch (IllegalArgumentException expectedException) {
-    }
-    try {
-      List<StructField> simpleFields = new ArrayList<StructField>();
-      simpleFields.add(DataType.createStructField("a", new DecimalType(), false));
-      simpleFields.add(DataType.createStructField("a", DataType.BooleanType, true));
-      simpleFields.add(DataType.createStructField("c", DataType.LongType, true));
-      DataType.createStructType(simpleFields);
-      Assert.fail();
-    } catch (IllegalArgumentException expectedException) {
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala
deleted file mode 100644
index e9740d9..0000000
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala
+++ /dev/null
@@ -1,88 +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.scalatest.FunSuite
-
-class DataTypeSuite extends FunSuite {
-
-  test("construct an ArrayType") {
-    val array = ArrayType(StringType)
-
-    assert(ArrayType(StringType, true) === array)
-  }
-
-  test("construct an MapType") {
-    val map = MapType(StringType, IntegerType)
-
-    assert(MapType(StringType, IntegerType, true) === map)
-  }
-
-  test("extract fields from a StructType") {
-    val struct = StructType(
-      StructField("a", IntegerType, true) ::
-      StructField("b", LongType, false) ::
-      StructField("c", StringType, true) ::
-      StructField("d", FloatType, true) :: Nil)
-
-    assert(StructField("b", LongType, false) === struct("b"))
-
-    intercept[IllegalArgumentException] {
-      struct("e")
-    }
-
-    val expectedStruct = StructType(
-      StructField("b", LongType, false) ::
-      StructField("d", FloatType, true) :: Nil)
-
-    assert(expectedStruct === struct(Set("b", "d")))
-    intercept[IllegalArgumentException] {
-      struct(Set("b", "d", "e", "f"))
-    }
-  }
-
-  def checkDataTypeJsonRepr(dataType: DataType): Unit = {
-    test(s"JSON - $dataType") {
-      assert(DataType.fromJson(dataType.json) === dataType)
-    }
-  }
-
-  checkDataTypeJsonRepr(BooleanType)
-  checkDataTypeJsonRepr(ByteType)
-  checkDataTypeJsonRepr(ShortType)
-  checkDataTypeJsonRepr(IntegerType)
-  checkDataTypeJsonRepr(LongType)
-  checkDataTypeJsonRepr(FloatType)
-  checkDataTypeJsonRepr(DoubleType)
-  checkDataTypeJsonRepr(DecimalType.Unlimited)
-  checkDataTypeJsonRepr(TimestampType)
-  checkDataTypeJsonRepr(StringType)
-  checkDataTypeJsonRepr(BinaryType)
-  checkDataTypeJsonRepr(ArrayType(DoubleType, true))
-  checkDataTypeJsonRepr(ArrayType(StringType, false))
-  checkDataTypeJsonRepr(MapType(IntegerType, StringType, true))
-  checkDataTypeJsonRepr(MapType(IntegerType, ArrayType(DoubleType), false))
-  val metadata = new MetadataBuilder()
-    .putString("name", "age")
-    .build()
-  checkDataTypeJsonRepr(
-    StructType(Seq(
-      StructField("a", IntegerType, nullable = true),
-      StructField("b", ArrayType(DoubleType), nullable = false),
-      StructField("c", DoubleType, nullable = false, metadata))))
-}


---------------------------------------------------------------------
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-5123][SQL] Reconcile Java/Scala API for data types.

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala
deleted file mode 100644
index 708362a..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala
+++ /dev/null
@@ -1,335 +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.catalyst.types.decimal
-
-import org.apache.spark.annotation.DeveloperApi
-
-/**
- * A mutable implementation of BigDecimal that can hold a Long if values are small enough.
- *
- * The semantics of the fields are as follows:
- * - _precision and _scale represent the SQL precision and scale we are looking for
- * - If decimalVal is set, it represents the whole decimal value
- * - Otherwise, the decimal value is longVal / (10 ** _scale)
- */
-final class Decimal extends Ordered[Decimal] with Serializable {
-  import Decimal.{MAX_LONG_DIGITS, POW_10, ROUNDING_MODE, BIG_DEC_ZERO}
-
-  private var decimalVal: BigDecimal = null
-  private var longVal: Long = 0L
-  private var _precision: Int = 1
-  private var _scale: Int = 0
-
-  def precision: Int = _precision
-  def scale: Int = _scale
-
-  /**
-   * Set this Decimal to the given Long. Will have precision 20 and scale 0.
-   */
-  def set(longVal: Long): Decimal = {
-    if (longVal <= -POW_10(MAX_LONG_DIGITS) || longVal >= POW_10(MAX_LONG_DIGITS)) {
-      // We can't represent this compactly as a long without risking overflow
-      this.decimalVal = BigDecimal(longVal)
-      this.longVal = 0L
-    } else {
-      this.decimalVal = null
-      this.longVal = longVal
-    }
-    this._precision = 20
-    this._scale = 0
-    this
-  }
-
-  /**
-   * Set this Decimal to the given Int. Will have precision 10 and scale 0.
-   */
-  def set(intVal: Int): Decimal = {
-    this.decimalVal = null
-    this.longVal = intVal
-    this._precision = 10
-    this._scale = 0
-    this
-  }
-
-  /**
-   * Set this Decimal to the given unscaled Long, with a given precision and scale.
-   */
-  def set(unscaled: Long, precision: Int, scale: Int): Decimal = {
-    if (setOrNull(unscaled, precision, scale) == null) {
-      throw new IllegalArgumentException("Unscaled value too large for precision")
-    }
-    this
-  }
-
-  /**
-   * Set this Decimal to the given unscaled Long, with a given precision and scale,
-   * and return it, or return null if it cannot be set due to overflow.
-   */
-  def setOrNull(unscaled: Long, precision: Int, scale: Int): Decimal = {
-    if (unscaled <= -POW_10(MAX_LONG_DIGITS) || unscaled >= POW_10(MAX_LONG_DIGITS)) {
-      // We can't represent this compactly as a long without risking overflow
-      if (precision < 19) {
-        return null  // Requested precision is too low to represent this value
-      }
-      this.decimalVal = BigDecimal(longVal)
-      this.longVal = 0L
-    } else {
-      val p = POW_10(math.min(precision, MAX_LONG_DIGITS))
-      if (unscaled <= -p || unscaled >= p) {
-        return null  // Requested precision is too low to represent this value
-      }
-      this.decimalVal = null
-      this.longVal = unscaled
-    }
-    this._precision = precision
-    this._scale = scale
-    this
-  }
-
-  /**
-   * Set this Decimal to the given BigDecimal value, with a given precision and scale.
-   */
-  def set(decimal: BigDecimal, precision: Int, scale: Int): Decimal = {
-    this.decimalVal = decimal.setScale(scale, ROUNDING_MODE)
-    require(decimalVal.precision <= precision, "Overflowed precision")
-    this.longVal = 0L
-    this._precision = precision
-    this._scale = scale
-    this
-  }
-
-  /**
-   * Set this Decimal to the given BigDecimal value, inheriting its precision and scale.
-   */
-  def set(decimal: BigDecimal): Decimal = {
-    this.decimalVal = decimal
-    this.longVal = 0L
-    this._precision = decimal.precision
-    this._scale = decimal.scale
-    this
-  }
-
-  /**
-   * Set this Decimal to the given Decimal value.
-   */
-  def set(decimal: Decimal): Decimal = {
-    this.decimalVal = decimal.decimalVal
-    this.longVal = decimal.longVal
-    this._precision = decimal._precision
-    this._scale = decimal._scale
-    this
-  }
-
-  def toBigDecimal: BigDecimal = {
-    if (decimalVal.ne(null)) {
-      decimalVal
-    } else {
-      BigDecimal(longVal, _scale)
-    }
-  }
-
-  def toUnscaledLong: Long = {
-    if (decimalVal.ne(null)) {
-      decimalVal.underlying().unscaledValue().longValue()
-    } else {
-      longVal
-    }
-  }
-
-  override def toString: String = toBigDecimal.toString()
-
-  @DeveloperApi
-  def toDebugString: String = {
-    if (decimalVal.ne(null)) {
-      s"Decimal(expanded,$decimalVal,$precision,$scale})"
-    } else {
-      s"Decimal(compact,$longVal,$precision,$scale})"
-    }
-  }
-
-  def toDouble: Double = toBigDecimal.doubleValue()
-
-  def toFloat: Float = toBigDecimal.floatValue()
-
-  def toLong: Long = {
-    if (decimalVal.eq(null)) {
-      longVal / POW_10(_scale)
-    } else {
-      decimalVal.longValue()
-    }
-  }
-
-  def toInt: Int = toLong.toInt
-
-  def toShort: Short = toLong.toShort
-
-  def toByte: Byte = toLong.toByte
-
-  /**
-   * Update precision and scale while keeping our value the same, and return true if successful.
-   *
-   * @return true if successful, false if overflow would occur
-   */
-  def changePrecision(precision: Int, scale: Int): Boolean = {
-    // First, update our longVal if we can, or transfer over to using a BigDecimal
-    if (decimalVal.eq(null)) {
-      if (scale < _scale) {
-        // Easier case: we just need to divide our scale down
-        val diff = _scale - scale
-        val droppedDigits = longVal % POW_10(diff)
-        longVal /= POW_10(diff)
-        if (math.abs(droppedDigits) * 2 >= POW_10(diff)) {
-          longVal += (if (longVal < 0) -1L else 1L)
-        }
-      } else if (scale > _scale) {
-        // We might be able to multiply longVal by a power of 10 and not overflow, but if not,
-        // switch to using a BigDecimal
-        val diff = scale - _scale
-        val p = POW_10(math.max(MAX_LONG_DIGITS - diff, 0))
-        if (diff <= MAX_LONG_DIGITS && longVal > -p && longVal < p) {
-          // Multiplying longVal by POW_10(diff) will still keep it below MAX_LONG_DIGITS
-          longVal *= POW_10(diff)
-        } else {
-          // Give up on using Longs; switch to BigDecimal, which we'll modify below
-          decimalVal = BigDecimal(longVal, _scale)
-        }
-      }
-      // In both cases, we will check whether our precision is okay below
-    }
-
-    if (decimalVal.ne(null)) {
-      // We get here if either we started with a BigDecimal, or we switched to one because we would
-      // have overflowed our Long; in either case we must rescale decimalVal to the new scale.
-      val newVal = decimalVal.setScale(scale, ROUNDING_MODE)
-      if (newVal.precision > precision) {
-        return false
-      }
-      decimalVal = newVal
-    } else {
-      // We're still using Longs, but we should check whether we match the new precision
-      val p = POW_10(math.min(_precision, MAX_LONG_DIGITS))
-      if (longVal <= -p || longVal >= p) {
-        // Note that we shouldn't have been able to fix this by switching to BigDecimal
-        return false
-      }
-    }
-
-    _precision = precision
-    _scale = scale
-    true
-  }
-
-  override def clone(): Decimal = new Decimal().set(this)
-
-  override def compare(other: Decimal): Int = {
-    if (decimalVal.eq(null) && other.decimalVal.eq(null) && _scale == other._scale) {
-      if (longVal < other.longVal) -1 else if (longVal == other.longVal) 0 else 1
-    } else {
-      toBigDecimal.compare(other.toBigDecimal)
-    }
-  }
-
-  override def equals(other: Any) = other match {
-    case d: Decimal =>
-      compare(d) == 0
-    case _ =>
-      false
-  }
-
-  override def hashCode(): Int = toBigDecimal.hashCode()
-
-  def isZero: Boolean = if (decimalVal.ne(null)) decimalVal == BIG_DEC_ZERO else longVal == 0
-
-  def + (that: Decimal): Decimal = Decimal(toBigDecimal + that.toBigDecimal)
-
-  def - (that: Decimal): Decimal = Decimal(toBigDecimal - that.toBigDecimal)
-
-  def * (that: Decimal): Decimal = Decimal(toBigDecimal * that.toBigDecimal)
-
-  def / (that: Decimal): Decimal =
-    if (that.isZero) null else Decimal(toBigDecimal / that.toBigDecimal)
-
-  def % (that: Decimal): Decimal =
-    if (that.isZero) null else Decimal(toBigDecimal % that.toBigDecimal)
-
-  def remainder(that: Decimal): Decimal = this % that
-
-  def unary_- : Decimal = {
-    if (decimalVal.ne(null)) {
-      Decimal(-decimalVal)
-    } else {
-      Decimal(-longVal, precision, scale)
-    }
-  }
-}
-
-object Decimal {
-  private val ROUNDING_MODE = BigDecimal.RoundingMode.HALF_UP
-
-  /** Maximum number of decimal digits a Long can represent */
-  val MAX_LONG_DIGITS = 18
-
-  private val POW_10 = Array.tabulate[Long](MAX_LONG_DIGITS + 1)(i => math.pow(10, i).toLong)
-
-  private val BIG_DEC_ZERO = BigDecimal(0)
-
-  def apply(value: Double): Decimal = new Decimal().set(value)
-
-  def apply(value: Long): Decimal = new Decimal().set(value)
-
-  def apply(value: Int): Decimal = new Decimal().set(value)
-
-  def apply(value: BigDecimal): Decimal = new Decimal().set(value)
-
-  def apply(value: BigDecimal, precision: Int, scale: Int): Decimal =
-    new Decimal().set(value, precision, scale)
-
-  def apply(unscaled: Long, precision: Int, scale: Int): Decimal =
-    new Decimal().set(unscaled, precision, scale)
-
-  def apply(value: String): Decimal = new Decimal().set(BigDecimal(value))
-
-  // Evidence parameters for Decimal considered either as Fractional or Integral. We provide two
-  // parameters inheriting from a common trait since both traits define mkNumericOps.
-  // See scala.math's Numeric.scala for examples for Scala's built-in types.
-
-  /** Common methods for Decimal evidence parameters */
-  trait DecimalIsConflicted extends Numeric[Decimal] {
-    override def plus(x: Decimal, y: Decimal): Decimal = x + y
-    override def times(x: Decimal, y: Decimal): Decimal = x * y
-    override def minus(x: Decimal, y: Decimal): Decimal = x - y
-    override def negate(x: Decimal): Decimal = -x
-    override def toDouble(x: Decimal): Double = x.toDouble
-    override def toFloat(x: Decimal): Float = x.toFloat
-    override def toInt(x: Decimal): Int = x.toInt
-    override def toLong(x: Decimal): Long = x.toLong
-    override def fromInt(x: Int): Decimal = new Decimal().set(x)
-    override def compare(x: Decimal, y: Decimal): Int = x.compare(y)
-  }
-
-  /** A [[scala.math.Fractional]] evidence parameter for Decimals. */
-  object DecimalIsFractional extends DecimalIsConflicted with Fractional[Decimal] {
-    override def div(x: Decimal, y: Decimal): Decimal = x / y
-  }
-
-  /** A [[scala.math.Integral]] evidence parameter for Decimals. */
-  object DecimalAsIfIntegral extends DecimalIsConflicted with Integral[Decimal] {
-    override def quot(x: Decimal, y: Decimal): Decimal = x / y
-    override def rem(x: Decimal, y: Decimal): Decimal = x % y
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala
deleted file mode 100644
index de24449..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala
+++ /dev/null
@@ -1,23 +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.catalyst
-/**
- * Contains a type system for attributes produced by relations, including complex types like
- * structs, arrays and maps.
- */
-package object types

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala
deleted file mode 100755
index 8172733..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala
+++ /dev/null
@@ -1,258 +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.catalyst.util
-
-import scala.collection.mutable
-
-import org.json4s._
-import org.json4s.jackson.JsonMethods._
-
-/**
- * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean,
- * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and
- * Array[Metadata]. JSON is used for serialization.
- *
- * The default constructor is private. User should use either [[MetadataBuilder]] or
- * [[Metadata$#fromJson]] to create Metadata instances.
- *
- * @param map an immutable map that stores the data
- */
-sealed class Metadata private[util] (private[util] val map: Map[String, Any]) extends Serializable {
-
-  /** Tests whether this Metadata contains a binding for a key. */
-  def contains(key: String): Boolean = map.contains(key)
-
-  /** Gets a Long. */
-  def getLong(key: String): Long = get(key)
-
-  /** Gets a Double. */
-  def getDouble(key: String): Double = get(key)
-
-  /** Gets a Boolean. */
-  def getBoolean(key: String): Boolean = get(key)
-
-  /** Gets a String. */
-  def getString(key: String): String = get(key)
-
-  /** Gets a Metadata. */
-  def getMetadata(key: String): Metadata = get(key)
-
-  /** Gets a Long array. */
-  def getLongArray(key: String): Array[Long] = get(key)
-
-  /** Gets a Double array. */
-  def getDoubleArray(key: String): Array[Double] = get(key)
-
-  /** Gets a Boolean array. */
-  def getBooleanArray(key: String): Array[Boolean] = get(key)
-
-  /** Gets a String array. */
-  def getStringArray(key: String): Array[String] = get(key)
-
-  /** Gets a Metadata array. */
-  def getMetadataArray(key: String): Array[Metadata] = get(key)
-
-  /** Converts to its JSON representation. */
-  def json: String = compact(render(jsonValue))
-
-  override def toString: String = json
-
-  override def equals(obj: Any): Boolean = {
-    obj match {
-      case that: Metadata =>
-        if (map.keySet == that.map.keySet) {
-          map.keys.forall { k =>
-            (map(k), that.map(k)) match {
-              case (v0: Array[_], v1: Array[_]) =>
-                v0.view == v1.view
-              case (v0, v1) =>
-                v0 == v1
-            }
-          }
-        } else {
-          false
-        }
-      case other =>
-        false
-    }
-  }
-
-  override def hashCode: Int = Metadata.hash(this)
-
-  private def get[T](key: String): T = {
-    map(key).asInstanceOf[T]
-  }
-
-  private[sql] def jsonValue: JValue = Metadata.toJsonValue(this)
-}
-
-object Metadata {
-
-  /** Returns an empty Metadata. */
-  def empty: Metadata = new Metadata(Map.empty)
-
-  /** Creates a Metadata instance from JSON. */
-  def fromJson(json: String): Metadata = {
-    fromJObject(parse(json).asInstanceOf[JObject])
-  }
-
-  /** Creates a Metadata instance from JSON AST. */
-  private[sql] def fromJObject(jObj: JObject): Metadata = {
-    val builder = new MetadataBuilder
-    jObj.obj.foreach {
-      case (key, JInt(value)) =>
-        builder.putLong(key, value.toLong)
-      case (key, JDouble(value)) =>
-        builder.putDouble(key, value)
-      case (key, JBool(value)) =>
-        builder.putBoolean(key, value)
-      case (key, JString(value)) =>
-        builder.putString(key, value)
-      case (key, o: JObject) =>
-        builder.putMetadata(key, fromJObject(o))
-      case (key, JArray(value)) =>
-        if (value.isEmpty) {
-          // If it is an empty array, we cannot infer its element type. We put an empty Array[Long].
-          builder.putLongArray(key, Array.empty)
-        } else {
-          value.head match {
-            case _: JInt =>
-              builder.putLongArray(key, value.asInstanceOf[List[JInt]].map(_.num.toLong).toArray)
-            case _: JDouble =>
-              builder.putDoubleArray(key, value.asInstanceOf[List[JDouble]].map(_.num).toArray)
-            case _: JBool =>
-              builder.putBooleanArray(key, value.asInstanceOf[List[JBool]].map(_.value).toArray)
-            case _: JString =>
-              builder.putStringArray(key, value.asInstanceOf[List[JString]].map(_.s).toArray)
-            case _: JObject =>
-              builder.putMetadataArray(
-                key, value.asInstanceOf[List[JObject]].map(fromJObject).toArray)
-            case other =>
-              throw new RuntimeException(s"Do not support array of type ${other.getClass}.")
-          }
-        }
-      case other =>
-        throw new RuntimeException(s"Do not support type ${other.getClass}.")
-    }
-    builder.build()
-  }
-
-  /** Converts to JSON AST. */
-  private def toJsonValue(obj: Any): JValue = {
-    obj match {
-      case map: Map[_, _] =>
-        val fields = map.toList.map { case (k: String, v) => (k, toJsonValue(v)) }
-        JObject(fields)
-      case arr: Array[_] =>
-        val values = arr.toList.map(toJsonValue)
-        JArray(values)
-      case x: Long =>
-        JInt(x)
-      case x: Double =>
-        JDouble(x)
-      case x: Boolean =>
-        JBool(x)
-      case x: String =>
-        JString(x)
-      case x: Metadata =>
-        toJsonValue(x.map)
-      case other =>
-        throw new RuntimeException(s"Do not support type ${other.getClass}.")
-    }
-  }
-
-  /** Computes the hash code for the types we support. */
-  private def hash(obj: Any): Int = {
-    obj match {
-      case map: Map[_, _] =>
-        map.mapValues(hash).##
-      case arr: Array[_] =>
-        // Seq.empty[T] has the same hashCode regardless of T.
-        arr.toSeq.map(hash).##
-      case x: Long =>
-        x.##
-      case x: Double =>
-        x.##
-      case x: Boolean =>
-        x.##
-      case x: String =>
-        x.##
-      case x: Metadata =>
-        hash(x.map)
-      case other =>
-        throw new RuntimeException(s"Do not support type ${other.getClass}.")
-    }
-  }
-}
-
-/**
- * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former.
- */
-class MetadataBuilder {
-
-  private val map: mutable.Map[String, Any] = mutable.Map.empty
-
-  /** Returns the immutable version of this map.  Used for java interop. */
-  protected def getMap = map.toMap
-
-  /** Include the content of an existing [[Metadata]] instance. */
-  def withMetadata(metadata: Metadata): this.type = {
-    map ++= metadata.map
-    this
-  }
-
-  /** Puts a Long. */
-  def putLong(key: String, value: Long): this.type = put(key, value)
-
-  /** Puts a Double. */
-  def putDouble(key: String, value: Double): this.type = put(key, value)
-
-  /** Puts a Boolean. */
-  def putBoolean(key: String, value: Boolean): this.type = put(key, value)
-
-  /** Puts a String. */
-  def putString(key: String, value: String): this.type = put(key, value)
-
-  /** Puts a [[Metadata]]. */
-  def putMetadata(key: String, value: Metadata): this.type = put(key, value)
-
-  /** Puts a Long array. */
-  def putLongArray(key: String, value: Array[Long]): this.type = put(key, value)
-
-  /** Puts a Double array. */
-  def putDoubleArray(key: String, value: Array[Double]): this.type = put(key, value)
-
-  /** Puts a Boolean array. */
-  def putBooleanArray(key: String, value: Array[Boolean]): this.type = put(key, value)
-
-  /** Puts a String array. */
-  def putStringArray(key: String, value: Array[String]): this.type = put(key, value)
-
-  /** Puts a [[Metadata]] array. */
-  def putMetadataArray(key: String, value: Array[Metadata]): this.type = put(key, value)
-
-  /** Builds the [[Metadata]] instance. */
-  def build(): Metadata = {
-    new Metadata(map.toMap)
-  }
-
-  private def put(key: String, value: Any): this.type = {
-    map.put(key, value)
-    this
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala
new file mode 100644
index 0000000..2a8914c
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala
@@ -0,0 +1,68 @@
+/*
+ * 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.types
+
+import java.text.SimpleDateFormat
+
+import org.apache.spark.sql.catalyst.ScalaReflection
+import org.apache.spark.sql.types.decimal.Decimal
+
+
+protected[sql] object DataTypeConversions {
+
+  def stringToTime(s: String): java.util.Date = {
+    if (!s.contains('T')) {
+      // JDBC escape string
+      if (s.contains(' ')) {
+        java.sql.Timestamp.valueOf(s)
+      } else {
+        java.sql.Date.valueOf(s)
+      }
+    } else if (s.endsWith("Z")) {
+      // this is zero timezone of ISO8601
+      stringToTime(s.substring(0, s.length - 1) + "GMT-00:00")
+    } else if (s.indexOf("GMT") == -1) {
+      // timezone with ISO8601
+      val inset = "+00.00".length
+      val s0 = s.substring(0, s.length - inset)
+      val s1 = s.substring(s.length - inset, s.length)
+      if (s0.substring(s0.lastIndexOf(':')).contains('.')) {
+        stringToTime(s0 + "GMT" + s1)
+      } else {
+        stringToTime(s0 + ".0GMT" + s1)
+      }
+    } else {
+      // ISO8601 with GMT insert
+      val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd'T'HH:mm:ss.SSSz" )
+      ISO8601GMT.parse(s)
+    }
+  }
+
+  /** Converts Java objects to catalyst rows / types */
+  def convertJavaToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match {
+    case (obj, udt: UserDefinedType[_]) => ScalaReflection.convertToCatalyst(obj, udt) // Scala type
+    case (d: java.math.BigDecimal, _) => Decimal(BigDecimal(d))
+    case (other, _) => other
+  }
+
+  /** Converts Java objects to catalyst rows / types */
+  def convertCatalystToJava(a: Any): Any = a match {
+    case d: scala.math.BigDecimal => d.underlying()
+    case other => other
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypes.java
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypes.java b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypes.java
new file mode 100644
index 0000000..e457542
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypes.java
@@ -0,0 +1,208 @@
+/*
+ * 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.types;
+
+import java.util.*;
+
+/**
+ * To get/create specific data type, users should use singleton objects and factory methods
+ * provided by this class.
+ */
+public class DataTypes {
+  /**
+   * Gets the StringType object.
+   */
+  public static final DataType StringType = StringType$.MODULE$;
+
+  /**
+   * Gets the BinaryType object.
+   */
+  public static final DataType BinaryType = BinaryType$.MODULE$;
+
+  /**
+   * Gets the BooleanType object.
+   */
+  public static final DataType BooleanType = BooleanType$.MODULE$;
+
+  /**
+   * Gets the DateType object.
+   */
+  public static final DataType DateType = DateType$.MODULE$;
+
+  /**
+   * Gets the TimestampType object.
+   */
+  public static final DataType TimestampType = TimestampType$.MODULE$;
+
+  /**
+   * Gets the DoubleType object.
+   */
+  public static final DataType DoubleType = DoubleType$.MODULE$;
+
+  /**
+   * Gets the FloatType object.
+   */
+  public static final DataType FloatType = FloatType$.MODULE$;
+
+  /**
+   * Gets the ByteType object.
+   */
+  public static final DataType ByteType = ByteType$.MODULE$;
+
+  /**
+   * Gets the IntegerType object.
+   */
+  public static final DataType IntegerType = IntegerType$.MODULE$;
+
+  /**
+   * Gets the LongType object.
+   */
+  public static final DataType LongType = LongType$.MODULE$;
+
+  /**
+   * Gets the ShortType object.
+   */
+  public static final DataType ShortType = ShortType$.MODULE$;
+
+  /**
+   * Gets the NullType object.
+   */
+  public static final DataType NullType = NullType$.MODULE$;
+
+  /**
+   * Creates an ArrayType by specifying the data type of elements ({@code elementType}).
+   * The field of {@code containsNull} is set to {@code true}.
+   */
+  public static ArrayType createArrayType(DataType elementType) {
+    if (elementType == null) {
+      throw new IllegalArgumentException("elementType should not be null.");
+    }
+    return new ArrayType(elementType, true);
+  }
+
+  /**
+   * Creates an ArrayType by specifying the data type of elements ({@code elementType}) and
+   * whether the array contains null values ({@code containsNull}).
+   */
+  public static ArrayType createArrayType(DataType elementType, boolean containsNull) {
+    if (elementType == null) {
+      throw new IllegalArgumentException("elementType should not be null.");
+    }
+    return new ArrayType(elementType, containsNull);
+  }
+
+  public static DecimalType createDecimalType(int precision, int scale) {
+    return DecimalType$.MODULE$.apply(precision, scale);
+  }
+
+  public static DecimalType createDecimalType() {
+    return DecimalType$.MODULE$.Unlimited();
+  }
+
+  /**
+   * Creates a MapType by specifying the data type of keys ({@code keyType}) and values
+   * ({@code keyType}). The field of {@code valueContainsNull} is set to {@code true}.
+   */
+  public static MapType createMapType(DataType keyType, DataType valueType) {
+    if (keyType == null) {
+      throw new IllegalArgumentException("keyType should not be null.");
+    }
+    if (valueType == null) {
+      throw new IllegalArgumentException("valueType should not be null.");
+    }
+    return new MapType(keyType, valueType, true);
+  }
+
+  /**
+   * Creates a MapType by specifying the data type of keys ({@code keyType}), the data type of
+   * values ({@code keyType}), and whether values contain any null value
+   * ({@code valueContainsNull}).
+   */
+  public static MapType createMapType(
+      DataType keyType,
+      DataType valueType,
+      boolean valueContainsNull) {
+    if (keyType == null) {
+      throw new IllegalArgumentException("keyType should not be null.");
+    }
+    if (valueType == null) {
+      throw new IllegalArgumentException("valueType should not be null.");
+    }
+    return new MapType(keyType, valueType, valueContainsNull);
+  }
+
+  /**
+   * Creates a StructField by specifying the name ({@code name}), data type ({@code dataType}) and
+   * whether values of this field can be null values ({@code nullable}).
+   */
+  public static StructField createStructField(
+      String name,
+      DataType dataType,
+      boolean nullable,
+      Metadata metadata) {
+    if (name == null) {
+      throw new IllegalArgumentException("name should not be null.");
+    }
+    if (dataType == null) {
+      throw new IllegalArgumentException("dataType should not be null.");
+    }
+    if (metadata == null) {
+      throw new IllegalArgumentException("metadata should not be null.");
+    }
+    return new StructField(name, dataType, nullable, metadata);
+  }
+
+  /**
+   * Creates a StructField with empty metadata.
+   *
+   * @see #createStructField(String, DataType, boolean, Metadata)
+   */
+  public static StructField createStructField(String name, DataType dataType, boolean nullable) {
+    return createStructField(name, dataType, nullable, (new MetadataBuilder()).build());
+  }
+
+  /**
+   * Creates a StructType with the given list of StructFields ({@code fields}).
+   */
+  public static StructType createStructType(List<StructField> fields) {
+    return createStructType(fields.toArray(new StructField[0]));
+  }
+
+  /**
+   * Creates a StructType with the given StructField array ({@code fields}).
+   */
+  public static StructType createStructType(StructField[] fields) {
+    if (fields == null) {
+      throw new IllegalArgumentException("fields should not be null.");
+    }
+    Set<String> distinctNames = new HashSet<String>();
+    for (StructField field : fields) {
+      if (field == null) {
+        throw new IllegalArgumentException(
+          "fields should not contain any null.");
+      }
+
+      distinctNames.add(field.name());
+    }
+    if (distinctNames.size() != fields.length) {
+      throw new IllegalArgumentException("fields should have distinct names.");
+    }
+
+    return StructType$.MODULE$.apply(fields);
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala
new file mode 100755
index 0000000..e50e976
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala
@@ -0,0 +1,268 @@
+/*
+ * 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.types
+
+import scala.collection.mutable
+
+import org.json4s._
+import org.json4s.jackson.JsonMethods._
+
+import org.apache.spark.annotation.DeveloperApi
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean,
+ * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and
+ * Array[Metadata]. JSON is used for serialization.
+ *
+ * The default constructor is private. User should use either [[MetadataBuilder]] or
+ * [[Metadata.fromJson()]] to create Metadata instances.
+ *
+ * @param map an immutable map that stores the data
+ */
+@DeveloperApi
+sealed class Metadata private[types] (private[types] val map: Map[String, Any])
+  extends Serializable {
+
+  /** Tests whether this Metadata contains a binding for a key. */
+  def contains(key: String): Boolean = map.contains(key)
+
+  /** Gets a Long. */
+  def getLong(key: String): Long = get(key)
+
+  /** Gets a Double. */
+  def getDouble(key: String): Double = get(key)
+
+  /** Gets a Boolean. */
+  def getBoolean(key: String): Boolean = get(key)
+
+  /** Gets a String. */
+  def getString(key: String): String = get(key)
+
+  /** Gets a Metadata. */
+  def getMetadata(key: String): Metadata = get(key)
+
+  /** Gets a Long array. */
+  def getLongArray(key: String): Array[Long] = get(key)
+
+  /** Gets a Double array. */
+  def getDoubleArray(key: String): Array[Double] = get(key)
+
+  /** Gets a Boolean array. */
+  def getBooleanArray(key: String): Array[Boolean] = get(key)
+
+  /** Gets a String array. */
+  def getStringArray(key: String): Array[String] = get(key)
+
+  /** Gets a Metadata array. */
+  def getMetadataArray(key: String): Array[Metadata] = get(key)
+
+  /** Converts to its JSON representation. */
+  def json: String = compact(render(jsonValue))
+
+  override def toString: String = json
+
+  override def equals(obj: Any): Boolean = {
+    obj match {
+      case that: Metadata =>
+        if (map.keySet == that.map.keySet) {
+          map.keys.forall { k =>
+            (map(k), that.map(k)) match {
+              case (v0: Array[_], v1: Array[_]) =>
+                v0.view == v1.view
+              case (v0, v1) =>
+                v0 == v1
+            }
+          }
+        } else {
+          false
+        }
+      case other =>
+        false
+    }
+  }
+
+  override def hashCode: Int = Metadata.hash(this)
+
+  private def get[T](key: String): T = {
+    map(key).asInstanceOf[T]
+  }
+
+  private[sql] def jsonValue: JValue = Metadata.toJsonValue(this)
+}
+
+object Metadata {
+
+  /** Returns an empty Metadata. */
+  def empty: Metadata = new Metadata(Map.empty)
+
+  /** Creates a Metadata instance from JSON. */
+  def fromJson(json: String): Metadata = {
+    fromJObject(parse(json).asInstanceOf[JObject])
+  }
+
+  /** Creates a Metadata instance from JSON AST. */
+  private[sql] def fromJObject(jObj: JObject): Metadata = {
+    val builder = new MetadataBuilder
+    jObj.obj.foreach {
+      case (key, JInt(value)) =>
+        builder.putLong(key, value.toLong)
+      case (key, JDouble(value)) =>
+        builder.putDouble(key, value)
+      case (key, JBool(value)) =>
+        builder.putBoolean(key, value)
+      case (key, JString(value)) =>
+        builder.putString(key, value)
+      case (key, o: JObject) =>
+        builder.putMetadata(key, fromJObject(o))
+      case (key, JArray(value)) =>
+        if (value.isEmpty) {
+          // If it is an empty array, we cannot infer its element type. We put an empty Array[Long].
+          builder.putLongArray(key, Array.empty)
+        } else {
+          value.head match {
+            case _: JInt =>
+              builder.putLongArray(key, value.asInstanceOf[List[JInt]].map(_.num.toLong).toArray)
+            case _: JDouble =>
+              builder.putDoubleArray(key, value.asInstanceOf[List[JDouble]].map(_.num).toArray)
+            case _: JBool =>
+              builder.putBooleanArray(key, value.asInstanceOf[List[JBool]].map(_.value).toArray)
+            case _: JString =>
+              builder.putStringArray(key, value.asInstanceOf[List[JString]].map(_.s).toArray)
+            case _: JObject =>
+              builder.putMetadataArray(
+                key, value.asInstanceOf[List[JObject]].map(fromJObject).toArray)
+            case other =>
+              throw new RuntimeException(s"Do not support array of type ${other.getClass}.")
+          }
+        }
+      case other =>
+        throw new RuntimeException(s"Do not support type ${other.getClass}.")
+    }
+    builder.build()
+  }
+
+  /** Converts to JSON AST. */
+  private def toJsonValue(obj: Any): JValue = {
+    obj match {
+      case map: Map[_, _] =>
+        val fields = map.toList.map { case (k: String, v) => (k, toJsonValue(v)) }
+        JObject(fields)
+      case arr: Array[_] =>
+        val values = arr.toList.map(toJsonValue)
+        JArray(values)
+      case x: Long =>
+        JInt(x)
+      case x: Double =>
+        JDouble(x)
+      case x: Boolean =>
+        JBool(x)
+      case x: String =>
+        JString(x)
+      case x: Metadata =>
+        toJsonValue(x.map)
+      case other =>
+        throw new RuntimeException(s"Do not support type ${other.getClass}.")
+    }
+  }
+
+  /** Computes the hash code for the types we support. */
+  private def hash(obj: Any): Int = {
+    obj match {
+      case map: Map[_, _] =>
+        map.mapValues(hash).##
+      case arr: Array[_] =>
+        // Seq.empty[T] has the same hashCode regardless of T.
+        arr.toSeq.map(hash).##
+      case x: Long =>
+        x.##
+      case x: Double =>
+        x.##
+      case x: Boolean =>
+        x.##
+      case x: String =>
+        x.##
+      case x: Metadata =>
+        hash(x.map)
+      case other =>
+        throw new RuntimeException(s"Do not support type ${other.getClass}.")
+    }
+  }
+}
+
+/**
+ * :: DeveloperApi ::
+ *
+ * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former.
+ */
+@DeveloperApi
+class MetadataBuilder {
+
+  private val map: mutable.Map[String, Any] = mutable.Map.empty
+
+  /** Returns the immutable version of this map.  Used for java interop. */
+  protected def getMap = map.toMap
+
+  /** Include the content of an existing [[Metadata]] instance. */
+  def withMetadata(metadata: Metadata): this.type = {
+    map ++= metadata.map
+    this
+  }
+
+  /** Puts a Long. */
+  def putLong(key: String, value: Long): this.type = put(key, value)
+
+  /** Puts a Double. */
+  def putDouble(key: String, value: Double): this.type = put(key, value)
+
+  /** Puts a Boolean. */
+  def putBoolean(key: String, value: Boolean): this.type = put(key, value)
+
+  /** Puts a String. */
+  def putString(key: String, value: String): this.type = put(key, value)
+
+  /** Puts a [[Metadata]]. */
+  def putMetadata(key: String, value: Metadata): this.type = put(key, value)
+
+  /** Puts a Long array. */
+  def putLongArray(key: String, value: Array[Long]): this.type = put(key, value)
+
+  /** Puts a Double array. */
+  def putDoubleArray(key: String, value: Array[Double]): this.type = put(key, value)
+
+  /** Puts a Boolean array. */
+  def putBooleanArray(key: String, value: Array[Boolean]): this.type = put(key, value)
+
+  /** Puts a String array. */
+  def putStringArray(key: String, value: Array[String]): this.type = put(key, value)
+
+  /** Puts a [[Metadata]] array. */
+  def putMetadataArray(key: String, value: Array[Metadata]): this.type = put(key, value)
+
+  /** Builds the [[Metadata]] instance. */
+  def build(): Metadata = {
+    new Metadata(map.toMap)
+  }
+
+  private def put(key: String, value: Any): this.type = {
+    map.put(key, value)
+    this
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/types/SQLUserDefinedType.java
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/SQLUserDefinedType.java b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/SQLUserDefinedType.java
new file mode 100644
index 0000000..a64d2bb
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/SQLUserDefinedType.java
@@ -0,0 +1,45 @@
+/*
+ * 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.types;
+
+import java.lang.annotation.*;
+
+import org.apache.spark.annotation.DeveloperApi;
+
+/**
+ * ::DeveloperApi::
+ * A user-defined type which can be automatically recognized by a SQLContext and registered.
+ *
+ * WARNING: This annotation will only work if both Java and Scala reflection return the same class
+ *          names (after erasure) for the UDT.  This will NOT be the case when, e.g., the UDT class
+ *          is enclosed in an object (a singleton).
+ *
+ * WARNING: UDTs are currently only supported from Scala.
+ */
+// TODO: Should I used @Documented ?
+@DeveloperApi
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.TYPE)
+public @interface SQLUserDefinedType {
+
+  /**
+   * Returns an instance of the UserDefinedType which can serialize and deserialize the user
+   * class to and from Catalyst built-in types.
+   */
+  Class<? extends UserDefinedType<?> > udt();
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala
new file mode 100644
index 0000000..fa0a355
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala
@@ -0,0 +1,900 @@
+/*
+ * 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.types
+
+import java.sql.{Date, Timestamp}
+
+import scala.math.Numeric.{FloatAsIfIntegral, DoubleAsIfIntegral}
+import scala.reflect.ClassTag
+import scala.reflect.runtime.universe.{TypeTag, runtimeMirror, typeTag}
+import scala.util.parsing.combinator.RegexParsers
+
+import org.json4s._
+import org.json4s.JsonAST.JValue
+import org.json4s.JsonDSL._
+import org.json4s.jackson.JsonMethods._
+
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.sql.catalyst.ScalaReflectionLock
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression}
+import org.apache.spark.sql.types.decimal._
+import org.apache.spark.util.Utils
+
+
+object DataType {
+  def fromJson(json: String): DataType = parseDataType(parse(json))
+
+  private object JSortedObject {
+    def unapplySeq(value: JValue): Option[List[(String, JValue)]] = value match {
+      case JObject(seq) => Some(seq.toList.sortBy(_._1))
+      case _ => None
+    }
+  }
+
+  // NOTE: Map fields must be sorted in alphabetical order to keep consistent with the Python side.
+  private def parseDataType(json: JValue): DataType = json match {
+    case JString(name) =>
+      PrimitiveType.nameToType(name)
+
+    case JSortedObject(
+        ("containsNull", JBool(n)),
+        ("elementType", t: JValue),
+        ("type", JString("array"))) =>
+      ArrayType(parseDataType(t), n)
+
+    case JSortedObject(
+        ("keyType", k: JValue),
+        ("type", JString("map")),
+        ("valueContainsNull", JBool(n)),
+        ("valueType", v: JValue)) =>
+      MapType(parseDataType(k), parseDataType(v), n)
+
+    case JSortedObject(
+        ("fields", JArray(fields)),
+        ("type", JString("struct"))) =>
+      StructType(fields.map(parseStructField))
+
+    case JSortedObject(
+        ("class", JString(udtClass)),
+        ("pyClass", _),
+        ("sqlType", _),
+        ("type", JString("udt"))) =>
+      Class.forName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]]
+  }
+
+  private def parseStructField(json: JValue): StructField = json match {
+    case JSortedObject(
+        ("metadata", metadata: JObject),
+        ("name", JString(name)),
+        ("nullable", JBool(nullable)),
+        ("type", dataType: JValue)) =>
+      StructField(name, parseDataType(dataType), nullable, Metadata.fromJObject(metadata))
+    // Support reading schema when 'metadata' is missing.
+    case JSortedObject(
+        ("name", JString(name)),
+        ("nullable", JBool(nullable)),
+        ("type", dataType: JValue)) =>
+      StructField(name, parseDataType(dataType), nullable)
+  }
+
+  @deprecated("Use DataType.fromJson instead", "1.2.0")
+  def fromCaseClassString(string: String): DataType = CaseClassStringParser(string)
+
+  private object CaseClassStringParser extends RegexParsers {
+    protected lazy val primitiveType: Parser[DataType] =
+      ( "StringType" ^^^ StringType
+      | "FloatType" ^^^ FloatType
+      | "IntegerType" ^^^ IntegerType
+      | "ByteType" ^^^ ByteType
+      | "ShortType" ^^^ ShortType
+      | "DoubleType" ^^^ DoubleType
+      | "LongType" ^^^ LongType
+      | "BinaryType" ^^^ BinaryType
+      | "BooleanType" ^^^ BooleanType
+      | "DateType" ^^^ DateType
+      | "DecimalType()" ^^^ DecimalType.Unlimited
+      | fixedDecimalType
+      | "TimestampType" ^^^ TimestampType
+      )
+
+    protected lazy val fixedDecimalType: Parser[DataType] =
+      ("DecimalType(" ~> "[0-9]+".r) ~ ("," ~> "[0-9]+".r <~ ")") ^^ {
+        case precision ~ scale => DecimalType(precision.toInt, scale.toInt)
+      }
+
+    protected lazy val arrayType: Parser[DataType] =
+      "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ {
+        case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull)
+      }
+
+    protected lazy val mapType: Parser[DataType] =
+      "MapType" ~> "(" ~> dataType ~ "," ~ dataType ~ "," ~ boolVal <~ ")" ^^ {
+        case t1 ~ _ ~ t2 ~ _ ~ valueContainsNull => MapType(t1, t2, valueContainsNull)
+      }
+
+    protected lazy val structField: Parser[StructField] =
+      ("StructField(" ~> "[a-zA-Z0-9_]*".r) ~ ("," ~> dataType) ~ ("," ~> boolVal <~ ")") ^^ {
+        case name ~ tpe ~ nullable  =>
+          StructField(name, tpe, nullable = nullable)
+      }
+
+    protected lazy val boolVal: Parser[Boolean] =
+      ( "true" ^^^ true
+      | "false" ^^^ false
+      )
+
+    protected lazy val structType: Parser[DataType] =
+      "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ {
+        case fields => StructType(fields)
+      }
+
+    protected lazy val dataType: Parser[DataType] =
+      ( arrayType
+      | mapType
+      | structType
+      | primitiveType
+      )
+
+    /**
+     * Parses a string representation of a DataType.
+     *
+     * TODO: Generate parser as pickler...
+     */
+    def apply(asString: String): DataType = parseAll(dataType, asString) match {
+      case Success(result, _) => result
+      case failure: NoSuccess =>
+        throw new IllegalArgumentException(s"Unsupported dataType: $asString, $failure")
+    }
+
+  }
+
+  protected[types] def buildFormattedString(
+      dataType: DataType,
+      prefix: String,
+      builder: StringBuilder): Unit = {
+    dataType match {
+      case array: ArrayType =>
+        array.buildFormattedString(prefix, builder)
+      case struct: StructType =>
+        struct.buildFormattedString(prefix, builder)
+      case map: MapType =>
+        map.buildFormattedString(prefix, builder)
+      case _ =>
+    }
+  }
+
+  /**
+   * Compares two types, ignoring nullability of ArrayType, MapType, StructType.
+   */
+  private[sql] def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = {
+    (left, right) match {
+      case (ArrayType(leftElementType, _), ArrayType(rightElementType, _)) =>
+        equalsIgnoreNullability(leftElementType, rightElementType)
+      case (MapType(leftKeyType, leftValueType, _), MapType(rightKeyType, rightValueType, _)) =>
+        equalsIgnoreNullability(leftKeyType, rightKeyType) &&
+        equalsIgnoreNullability(leftValueType, rightValueType)
+      case (StructType(leftFields), StructType(rightFields)) =>
+        leftFields.size == rightFields.size &&
+        leftFields.zip(rightFields)
+          .forall{
+            case (left, right) =>
+              left.name == right.name && equalsIgnoreNullability(left.dataType, right.dataType)
+          }
+      case (left, right) => left == right
+    }
+  }
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The base type of all Spark SQL data types.
+ *
+ * @group dataType
+ */
+@DeveloperApi
+abstract class DataType {
+  /** Matches any expression that evaluates to this DataType */
+  def unapply(a: Expression): Boolean = a match {
+    case e: Expression if e.dataType == this => true
+    case _ => false
+  }
+
+  def isPrimitive: Boolean = false
+
+  def typeName: String = this.getClass.getSimpleName.stripSuffix("$").dropRight(4).toLowerCase
+
+  private[sql] def jsonValue: JValue = typeName
+
+  def json: String = compact(render(jsonValue))
+
+  def prettyJson: String = pretty(render(jsonValue))
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `NULL` values. Please use the singleton [[DataTypes.NullType]].
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case object NullType extends DataType
+
+
+object NativeType {
+  val all = Seq(
+    IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType)
+
+  def unapply(dt: DataType): Boolean = all.contains(dt)
+
+  val defaultSizeOf: Map[NativeType, Int] = Map(
+    IntegerType -> 4,
+    BooleanType -> 1,
+    LongType -> 8,
+    DoubleType -> 8,
+    FloatType -> 4,
+    ShortType -> 2,
+    ByteType -> 1,
+    StringType -> 4096)
+}
+
+
+trait PrimitiveType extends DataType {
+  override def isPrimitive = true
+}
+
+
+object PrimitiveType {
+  private val nonDecimals = Seq(NullType, DateType, TimestampType, BinaryType) ++ NativeType.all
+  private val nonDecimalNameToType = nonDecimals.map(t => t.typeName -> t).toMap
+
+  /** Given the string representation of a type, return its DataType */
+  private[sql] def nameToType(name: String): DataType = {
+    val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\d+)\s*\)""".r
+    name match {
+      case "decimal" => DecimalType.Unlimited
+      case FIXED_DECIMAL(precision, scale) => DecimalType(precision.toInt, scale.toInt)
+      case other => nonDecimalNameToType(other)
+    }
+  }
+}
+
+abstract class NativeType extends DataType {
+  private[sql] type JvmType
+  @transient private[sql] val tag: TypeTag[JvmType]
+  private[sql] val ordering: Ordering[JvmType]
+
+  @transient private[sql] val classTag = ScalaReflectionLock.synchronized {
+    val mirror = runtimeMirror(Utils.getSparkClassLoader)
+    ClassTag[JvmType](mirror.runtimeClass(tag.tpe))
+  }
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `String` values. Please use the singleton [[DataTypes.StringType]].
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case object StringType extends NativeType with PrimitiveType {
+  private[sql] type JvmType = String
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+  private[sql] val ordering = implicitly[Ordering[JvmType]]
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `Array[Byte]` values.
+ * Please use the singleton [[DataTypes.BinaryType]].
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case object BinaryType extends NativeType with PrimitiveType {
+  private[sql] type JvmType = Array[Byte]
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+  private[sql] val ordering = new Ordering[JvmType] {
+    def compare(x: Array[Byte], y: Array[Byte]): Int = {
+      for (i <- 0 until x.length; if i < y.length) {
+        val res = x(i).compareTo(y(i))
+        if (res != 0) return res
+      }
+      x.length - y.length
+    }
+  }
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `Boolean` values. Please use the singleton [[DataTypes.BooleanType]].
+ *
+ *@group dataType
+ */
+@DeveloperApi
+case object BooleanType extends NativeType with PrimitiveType {
+  private[sql] type JvmType = Boolean
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+  private[sql] val ordering = implicitly[Ordering[JvmType]]
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `java.sql.Timestamp` values.
+ * Please use the singleton [[DataTypes.TimestampType]].
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case object TimestampType extends NativeType {
+  private[sql] type JvmType = Timestamp
+
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+
+  private[sql] val ordering = new Ordering[JvmType] {
+    def compare(x: Timestamp, y: Timestamp) = x.compareTo(y)
+  }
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `java.sql.Date` values.
+ * Please use the singleton [[DataTypes.DateType]].
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case object DateType extends NativeType {
+  private[sql] type JvmType = Date
+
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+
+  private[sql] val ordering = new Ordering[JvmType] {
+    def compare(x: Date, y: Date) = x.compareTo(y)
+  }
+}
+
+
+abstract class NumericType extends NativeType with PrimitiveType {
+  // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for
+  // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a
+  // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets
+  // desugared by the compiler into an argument to the objects constructor. This means there is no
+  // longer an no argument constructor and thus the JVM cannot serialize the object anymore.
+  private[sql] val numeric: Numeric[JvmType]
+}
+
+
+object NumericType {
+  def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[NumericType]
+}
+
+
+/** Matcher for any expressions that evaluate to [[IntegralType]]s */
+object IntegralType {
+  def unapply(a: Expression): Boolean = a match {
+    case e: Expression if e.dataType.isInstanceOf[IntegralType] => true
+    case _ => false
+  }
+}
+
+
+sealed abstract class IntegralType extends NumericType {
+  private[sql] val integral: Integral[JvmType]
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `Long` values. Please use the singleton [[DataTypes.LongType]].
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case object LongType extends IntegralType {
+  private[sql] type JvmType = Long
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+  private[sql] val numeric = implicitly[Numeric[Long]]
+  private[sql] val integral = implicitly[Integral[Long]]
+  private[sql] val ordering = implicitly[Ordering[JvmType]]
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `Int` values. Please use the singleton [[DataTypes.IntegerType]].
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case object IntegerType extends IntegralType {
+  private[sql] type JvmType = Int
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+  private[sql] val numeric = implicitly[Numeric[Int]]
+  private[sql] val integral = implicitly[Integral[Int]]
+  private[sql] val ordering = implicitly[Ordering[JvmType]]
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `Short` values. Please use the singleton [[DataTypes.ShortType]].
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case object ShortType extends IntegralType {
+  private[sql] type JvmType = Short
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+  private[sql] val numeric = implicitly[Numeric[Short]]
+  private[sql] val integral = implicitly[Integral[Short]]
+  private[sql] val ordering = implicitly[Ordering[JvmType]]
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `Byte` values. Please use the singleton [[DataTypes.ByteType]].
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case object ByteType extends IntegralType {
+  private[sql] type JvmType = Byte
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+  private[sql] val numeric = implicitly[Numeric[Byte]]
+  private[sql] val integral = implicitly[Integral[Byte]]
+  private[sql] val ordering = implicitly[Ordering[JvmType]]
+}
+
+
+/** Matcher for any expressions that evaluate to [[FractionalType]]s */
+object FractionalType {
+  def unapply(a: Expression): Boolean = a match {
+    case e: Expression if e.dataType.isInstanceOf[FractionalType] => true
+    case _ => false
+  }
+}
+
+
+sealed abstract class FractionalType extends NumericType {
+  private[sql] val fractional: Fractional[JvmType]
+  private[sql] val asIntegral: Integral[JvmType]
+}
+
+
+/** Precision parameters for a Decimal */
+case class PrecisionInfo(precision: Int, scale: Int)
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `scala.math.BigDecimal` values.
+ * A Decimal that might have fixed precision and scale, or unlimited values for these.
+ *
+ * Please use [[DataTypes.createDecimalType()]] to create a specific instance.
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalType {
+  private[sql] type JvmType = Decimal
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+  private[sql] val numeric = Decimal.DecimalIsFractional
+  private[sql] val fractional = Decimal.DecimalIsFractional
+  private[sql] val ordering = Decimal.DecimalIsFractional
+  private[sql] val asIntegral = Decimal.DecimalAsIfIntegral
+
+  def precision: Int = precisionInfo.map(_.precision).getOrElse(-1)
+
+  def scale: Int = precisionInfo.map(_.scale).getOrElse(-1)
+
+  override def typeName: String = precisionInfo match {
+    case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)"
+    case None => "decimal"
+  }
+
+  override def toString: String = precisionInfo match {
+    case Some(PrecisionInfo(precision, scale)) => s"DecimalType($precision,$scale)"
+    case None => "DecimalType()"
+  }
+}
+
+
+/** Extra factory methods and pattern matchers for Decimals */
+object DecimalType {
+  val Unlimited: DecimalType = DecimalType(None)
+
+  object Fixed {
+    def unapply(t: DecimalType): Option[(Int, Int)] =
+      t.precisionInfo.map(p => (p.precision, p.scale))
+  }
+
+  object Expression {
+    def unapply(e: Expression): Option[(Int, Int)] = e.dataType match {
+      case t: DecimalType => t.precisionInfo.map(p => (p.precision, p.scale))
+      case _ => None
+    }
+  }
+
+  def apply(): DecimalType = Unlimited
+
+  def apply(precision: Int, scale: Int): DecimalType =
+    DecimalType(Some(PrecisionInfo(precision, scale)))
+
+  def unapply(t: DataType): Boolean = t.isInstanceOf[DecimalType]
+
+  def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[DecimalType]
+
+  def isFixed(dataType: DataType): Boolean = dataType match {
+    case DecimalType.Fixed(_, _) => true
+    case _ => false
+  }
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `Double` values. Please use the singleton [[DataTypes.DoubleType]].
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case object DoubleType extends FractionalType {
+  private[sql] type JvmType = Double
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+  private[sql] val numeric = implicitly[Numeric[Double]]
+  private[sql] val fractional = implicitly[Fractional[Double]]
+  private[sql] val ordering = implicitly[Ordering[JvmType]]
+  private[sql] val asIntegral = DoubleAsIfIntegral
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type representing `Float` values. Please use the singleton [[DataTypes.FloatType]].
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case object FloatType extends FractionalType {
+  private[sql] type JvmType = Float
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+  private[sql] val numeric = implicitly[Numeric[Float]]
+  private[sql] val fractional = implicitly[Fractional[Float]]
+  private[sql] val ordering = implicitly[Ordering[JvmType]]
+  private[sql] val asIntegral = FloatAsIfIntegral
+}
+
+
+object ArrayType {
+  /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */
+  def apply(elementType: DataType): ArrayType = ArrayType(elementType, true)
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type for collections of multiple values.
+ * Internally these are represented as columns that contain a ``scala.collection.Seq``.
+ *
+ * Please use [[DataTypes.createArrayType()]] to create a specific instance.
+ *
+ * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and
+ * `containsNull: Boolean`. The field of `elementType` is used to specify the type of
+ * array elements. The field of `containsNull` is used to specify if the array has `null` values.
+ *
+ * @param elementType The data type of values.
+ * @param containsNull Indicates if values have `null` values
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType {
+  private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
+    builder.append(
+      s"$prefix-- element: ${elementType.typeName} (containsNull = $containsNull)\n")
+    DataType.buildFormattedString(elementType, s"$prefix    |", builder)
+  }
+
+  override private[sql] def jsonValue =
+    ("type" -> typeName) ~
+      ("elementType" -> elementType.jsonValue) ~
+      ("containsNull" -> containsNull)
+}
+
+
+/**
+ * A field inside a StructType.
+ *
+ * @param name The name of this field.
+ * @param dataType The data type of this field.
+ * @param nullable Indicates if values of this field can be `null` values.
+ * @param metadata The metadata of this field. The metadata should be preserved during
+ *                 transformation if the content of the column is not modified, e.g, in selection.
+ */
+case class StructField(
+    name: String,
+    dataType: DataType,
+    nullable: Boolean = true,
+    metadata: Metadata = Metadata.empty) {
+
+  private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
+    builder.append(s"$prefix-- $name: ${dataType.typeName} (nullable = $nullable)\n")
+    DataType.buildFormattedString(dataType, s"$prefix    |", builder)
+  }
+
+  // override the default toString to be compatible with legacy parquet files.
+  override def toString: String = s"StructField($name,$dataType,$nullable)"
+
+  private[sql] def jsonValue: JValue = {
+    ("name" -> name) ~
+      ("type" -> dataType.jsonValue) ~
+      ("nullable" -> nullable) ~
+      ("metadata" -> metadata.jsonValue)
+  }
+}
+
+
+object StructType {
+  protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType =
+    StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))
+
+  def apply(fields: Seq[StructField]): StructType = StructType(fields.toArray)
+
+  def apply(fields: java.util.List[StructField]): StructType = {
+    StructType(fields.toArray.asInstanceOf[Array[StructField]])
+  }
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * A [[StructType]] object can be constructed by
+ * {{{
+ * StructType(fields: Seq[StructField])
+ * }}}
+ * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names.
+ * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned.
+ * If a provided name does not have a matching field, it will be ignored. For the case
+ * of extracting a single StructField, a `null` will be returned.
+ * Example:
+ * {{{
+ * import org.apache.spark.sql._
+ *
+ * val struct =
+ *   StructType(
+ *     StructField("a", IntegerType, true) ::
+ *     StructField("b", LongType, false) ::
+ *     StructField("c", BooleanType, false) :: Nil)
+ *
+ * // Extract a single StructField.
+ * val singleField = struct("b")
+ * // singleField: StructField = StructField(b,LongType,false)
+ *
+ * // This struct does not have a field called "d". null will be returned.
+ * val nonExisting = struct("d")
+ * // nonExisting: StructField = null
+ *
+ * // Extract multiple StructFields. Field names are provided in a set.
+ * // A StructType object will be returned.
+ * val twoFields = struct(Set("b", "c"))
+ * // twoFields: StructType =
+ * //   StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false)))
+ *
+ * // Any names without matching fields will be ignored.
+ * // For the case shown below, "d" will be ignored and
+ * // it is treated as struct(Set("b", "c")).
+ * val ignoreNonExisting = struct(Set("b", "c", "d"))
+ * // ignoreNonExisting: StructType =
+ * //   StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false)))
+ * }}}
+ *
+ * A [[org.apache.spark.sql.catalyst.expressions.Row]] object is used as a value of the StructType.
+ * Example:
+ * {{{
+ * import org.apache.spark.sql._
+ *
+ * val innerStruct =
+ *   StructType(
+ *     StructField("f1", IntegerType, true) ::
+ *     StructField("f2", LongType, false) ::
+ *     StructField("f3", BooleanType, false) :: Nil)
+ *
+ * val struct = StructType(
+ *   StructField("a", innerStruct, true) :: Nil)
+ *
+ * // Create a Row with the schema defined by struct
+ * val row = Row(Row(1, 2, true))
+ * // row: Row = [[1,2,true]]
+ * }}}
+ *
+ * @group dataType
+ */
+@DeveloperApi
+case class StructType(fields: Array[StructField]) extends DataType with Seq[StructField] {
+
+  /** Returns all field names in an array. */
+  def fieldNames: Array[String] = fields.map(_.name)
+
+  private lazy val fieldNamesSet: Set[String] = fieldNames.toSet
+  private lazy val nameToField: Map[String, StructField] = fields.map(f => f.name -> f).toMap
+
+  /**
+   * Extracts a [[StructField]] of the given name. If the [[StructType]] object does not
+   * have a name matching the given name, `null` will be returned.
+   */
+  def apply(name: String): StructField = {
+    nameToField.getOrElse(name, throw new IllegalArgumentException(s"Field $name does not exist."))
+  }
+
+  /**
+   * Returns a [[StructType]] containing [[StructField]]s of the given names, preserving the
+   * original order of fields. Those names which do not have matching fields will be ignored.
+   */
+  def apply(names: Set[String]): StructType = {
+    val nonExistFields = names -- fieldNamesSet
+    if (nonExistFields.nonEmpty) {
+      throw new IllegalArgumentException(
+        s"Field ${nonExistFields.mkString(",")} does not exist.")
+    }
+    // Preserve the original order of fields.
+    StructType(fields.filter(f => names.contains(f.name)))
+  }
+
+  protected[sql] def toAttributes: Seq[AttributeReference] =
+    map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)())
+
+  def treeString: String = {
+    val builder = new StringBuilder
+    builder.append("root\n")
+    val prefix = " |"
+    fields.foreach(field => field.buildFormattedString(prefix, builder))
+
+    builder.toString()
+  }
+
+  def printTreeString(): Unit = println(treeString)
+
+  private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
+    fields.foreach(field => field.buildFormattedString(prefix, builder))
+  }
+
+  override private[sql] def jsonValue =
+    ("type" -> typeName) ~
+      ("fields" -> map(_.jsonValue))
+
+  override def apply(fieldIndex: Int): StructField = fields(fieldIndex)
+
+  override def length: Int = fields.length
+
+  override def iterator: Iterator[StructField] = fields.iterator
+}
+
+
+object MapType {
+  /**
+   * Construct a [[MapType]] object with the given key type and value type.
+   * The `valueContainsNull` is true.
+   */
+  def apply(keyType: DataType, valueType: DataType): MapType =
+    MapType(keyType: DataType, valueType: DataType, valueContainsNull = true)
+}
+
+
+/**
+ * :: DeveloperApi ::
+ *
+ * The data type for Maps. Keys in a map are not allowed to have `null` values.
+ *
+ * Please use [[DataTypes.createMapType()]] to create a specific instance.
+ *
+ * @param keyType The data type of map keys.
+ * @param valueType The data type of map values.
+ * @param valueContainsNull Indicates if map values have `null` values.
+ *
+ * @group dataType
+ */
+case class MapType(
+    keyType: DataType,
+    valueType: DataType,
+    valueContainsNull: Boolean) extends DataType {
+  private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
+    builder.append(s"$prefix-- key: ${keyType.typeName}\n")
+    builder.append(s"$prefix-- value: ${valueType.typeName} " +
+      s"(valueContainsNull = $valueContainsNull)\n")
+    DataType.buildFormattedString(keyType, s"$prefix    |", builder)
+    DataType.buildFormattedString(valueType, s"$prefix    |", builder)
+  }
+
+  override private[sql] def jsonValue: JValue =
+    ("type" -> typeName) ~
+      ("keyType" -> keyType.jsonValue) ~
+      ("valueType" -> valueType.jsonValue) ~
+      ("valueContainsNull" -> valueContainsNull)
+}
+
+
+/**
+ * ::DeveloperApi::
+ * The data type for User Defined Types (UDTs).
+ *
+ * This interface allows a user to make their own classes more interoperable with SparkSQL;
+ * e.g., by creating a [[UserDefinedType]] for a class X, it becomes possible to create
+ * a SchemaRDD which has class X in the schema.
+ *
+ * For SparkSQL to recognize UDTs, the UDT must be annotated with
+ * [[SQLUserDefinedType]].
+ *
+ * The conversion via `serialize` occurs when instantiating a `SchemaRDD` from another RDD.
+ * The conversion via `deserialize` occurs when reading from a `SchemaRDD`.
+ */
+@DeveloperApi
+abstract class UserDefinedType[UserType] extends DataType with Serializable {
+
+  /** Underlying storage type for this UDT */
+  def sqlType: DataType
+
+  /** Paired Python UDT class, if exists. */
+  def pyUDT: String = null
+
+  /**
+   * Convert the user type to a SQL datum
+   *
+   * TODO: Can we make this take obj: UserType?  The issue is in ScalaReflection.convertToCatalyst,
+   *       where we need to convert Any to UserType.
+   */
+  def serialize(obj: Any): Any
+
+  /** Convert a SQL datum to the user type */
+  def deserialize(datum: Any): UserType
+
+  override private[sql] def jsonValue: JValue = {
+    ("type" -> "udt") ~
+      ("class" -> this.getClass.getName) ~
+      ("pyClass" -> pyUDT) ~
+      ("sqlType" -> sqlType.jsonValue)
+  }
+
+  /**
+   * Class object for the UserType
+   */
+  def userClass: java.lang.Class[UserType]
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/types/decimal/Decimal.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/decimal/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/decimal/Decimal.scala
new file mode 100644
index 0000000..c7864d1
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/decimal/Decimal.scala
@@ -0,0 +1,335 @@
+/*
+ * 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.types.decimal
+
+import org.apache.spark.annotation.DeveloperApi
+
+/**
+ * A mutable implementation of BigDecimal that can hold a Long if values are small enough.
+ *
+ * The semantics of the fields are as follows:
+ * - _precision and _scale represent the SQL precision and scale we are looking for
+ * - If decimalVal is set, it represents the whole decimal value
+ * - Otherwise, the decimal value is longVal / (10 ** _scale)
+ */
+final class Decimal extends Ordered[Decimal] with Serializable {
+  import Decimal.{MAX_LONG_DIGITS, POW_10, ROUNDING_MODE, BIG_DEC_ZERO}
+
+  private var decimalVal: BigDecimal = null
+  private var longVal: Long = 0L
+  private var _precision: Int = 1
+  private var _scale: Int = 0
+
+  def precision: Int = _precision
+  def scale: Int = _scale
+
+  /**
+   * Set this Decimal to the given Long. Will have precision 20 and scale 0.
+   */
+  def set(longVal: Long): Decimal = {
+    if (longVal <= -POW_10(MAX_LONG_DIGITS) || longVal >= POW_10(MAX_LONG_DIGITS)) {
+      // We can't represent this compactly as a long without risking overflow
+      this.decimalVal = BigDecimal(longVal)
+      this.longVal = 0L
+    } else {
+      this.decimalVal = null
+      this.longVal = longVal
+    }
+    this._precision = 20
+    this._scale = 0
+    this
+  }
+
+  /**
+   * Set this Decimal to the given Int. Will have precision 10 and scale 0.
+   */
+  def set(intVal: Int): Decimal = {
+    this.decimalVal = null
+    this.longVal = intVal
+    this._precision = 10
+    this._scale = 0
+    this
+  }
+
+  /**
+   * Set this Decimal to the given unscaled Long, with a given precision and scale.
+   */
+  def set(unscaled: Long, precision: Int, scale: Int): Decimal = {
+    if (setOrNull(unscaled, precision, scale) == null) {
+      throw new IllegalArgumentException("Unscaled value too large for precision")
+    }
+    this
+  }
+
+  /**
+   * Set this Decimal to the given unscaled Long, with a given precision and scale,
+   * and return it, or return null if it cannot be set due to overflow.
+   */
+  def setOrNull(unscaled: Long, precision: Int, scale: Int): Decimal = {
+    if (unscaled <= -POW_10(MAX_LONG_DIGITS) || unscaled >= POW_10(MAX_LONG_DIGITS)) {
+      // We can't represent this compactly as a long without risking overflow
+      if (precision < 19) {
+        return null  // Requested precision is too low to represent this value
+      }
+      this.decimalVal = BigDecimal(longVal)
+      this.longVal = 0L
+    } else {
+      val p = POW_10(math.min(precision, MAX_LONG_DIGITS))
+      if (unscaled <= -p || unscaled >= p) {
+        return null  // Requested precision is too low to represent this value
+      }
+      this.decimalVal = null
+      this.longVal = unscaled
+    }
+    this._precision = precision
+    this._scale = scale
+    this
+  }
+
+  /**
+   * Set this Decimal to the given BigDecimal value, with a given precision and scale.
+   */
+  def set(decimal: BigDecimal, precision: Int, scale: Int): Decimal = {
+    this.decimalVal = decimal.setScale(scale, ROUNDING_MODE)
+    require(decimalVal.precision <= precision, "Overflowed precision")
+    this.longVal = 0L
+    this._precision = precision
+    this._scale = scale
+    this
+  }
+
+  /**
+   * Set this Decimal to the given BigDecimal value, inheriting its precision and scale.
+   */
+  def set(decimal: BigDecimal): Decimal = {
+    this.decimalVal = decimal
+    this.longVal = 0L
+    this._precision = decimal.precision
+    this._scale = decimal.scale
+    this
+  }
+
+  /**
+   * Set this Decimal to the given Decimal value.
+   */
+  def set(decimal: Decimal): Decimal = {
+    this.decimalVal = decimal.decimalVal
+    this.longVal = decimal.longVal
+    this._precision = decimal._precision
+    this._scale = decimal._scale
+    this
+  }
+
+  def toBigDecimal: BigDecimal = {
+    if (decimalVal.ne(null)) {
+      decimalVal
+    } else {
+      BigDecimal(longVal, _scale)
+    }
+  }
+
+  def toUnscaledLong: Long = {
+    if (decimalVal.ne(null)) {
+      decimalVal.underlying().unscaledValue().longValue()
+    } else {
+      longVal
+    }
+  }
+
+  override def toString: String = toBigDecimal.toString()
+
+  @DeveloperApi
+  def toDebugString: String = {
+    if (decimalVal.ne(null)) {
+      s"Decimal(expanded,$decimalVal,$precision,$scale})"
+    } else {
+      s"Decimal(compact,$longVal,$precision,$scale})"
+    }
+  }
+
+  def toDouble: Double = toBigDecimal.doubleValue()
+
+  def toFloat: Float = toBigDecimal.floatValue()
+
+  def toLong: Long = {
+    if (decimalVal.eq(null)) {
+      longVal / POW_10(_scale)
+    } else {
+      decimalVal.longValue()
+    }
+  }
+
+  def toInt: Int = toLong.toInt
+
+  def toShort: Short = toLong.toShort
+
+  def toByte: Byte = toLong.toByte
+
+  /**
+   * Update precision and scale while keeping our value the same, and return true if successful.
+   *
+   * @return true if successful, false if overflow would occur
+   */
+  def changePrecision(precision: Int, scale: Int): Boolean = {
+    // First, update our longVal if we can, or transfer over to using a BigDecimal
+    if (decimalVal.eq(null)) {
+      if (scale < _scale) {
+        // Easier case: we just need to divide our scale down
+        val diff = _scale - scale
+        val droppedDigits = longVal % POW_10(diff)
+        longVal /= POW_10(diff)
+        if (math.abs(droppedDigits) * 2 >= POW_10(diff)) {
+          longVal += (if (longVal < 0) -1L else 1L)
+        }
+      } else if (scale > _scale) {
+        // We might be able to multiply longVal by a power of 10 and not overflow, but if not,
+        // switch to using a BigDecimal
+        val diff = scale - _scale
+        val p = POW_10(math.max(MAX_LONG_DIGITS - diff, 0))
+        if (diff <= MAX_LONG_DIGITS && longVal > -p && longVal < p) {
+          // Multiplying longVal by POW_10(diff) will still keep it below MAX_LONG_DIGITS
+          longVal *= POW_10(diff)
+        } else {
+          // Give up on using Longs; switch to BigDecimal, which we'll modify below
+          decimalVal = BigDecimal(longVal, _scale)
+        }
+      }
+      // In both cases, we will check whether our precision is okay below
+    }
+
+    if (decimalVal.ne(null)) {
+      // We get here if either we started with a BigDecimal, or we switched to one because we would
+      // have overflowed our Long; in either case we must rescale decimalVal to the new scale.
+      val newVal = decimalVal.setScale(scale, ROUNDING_MODE)
+      if (newVal.precision > precision) {
+        return false
+      }
+      decimalVal = newVal
+    } else {
+      // We're still using Longs, but we should check whether we match the new precision
+      val p = POW_10(math.min(_precision, MAX_LONG_DIGITS))
+      if (longVal <= -p || longVal >= p) {
+        // Note that we shouldn't have been able to fix this by switching to BigDecimal
+        return false
+      }
+    }
+
+    _precision = precision
+    _scale = scale
+    true
+  }
+
+  override def clone(): Decimal = new Decimal().set(this)
+
+  override def compare(other: Decimal): Int = {
+    if (decimalVal.eq(null) && other.decimalVal.eq(null) && _scale == other._scale) {
+      if (longVal < other.longVal) -1 else if (longVal == other.longVal) 0 else 1
+    } else {
+      toBigDecimal.compare(other.toBigDecimal)
+    }
+  }
+
+  override def equals(other: Any) = other match {
+    case d: Decimal =>
+      compare(d) == 0
+    case _ =>
+      false
+  }
+
+  override def hashCode(): Int = toBigDecimal.hashCode()
+
+  def isZero: Boolean = if (decimalVal.ne(null)) decimalVal == BIG_DEC_ZERO else longVal == 0
+
+  def + (that: Decimal): Decimal = Decimal(toBigDecimal + that.toBigDecimal)
+
+  def - (that: Decimal): Decimal = Decimal(toBigDecimal - that.toBigDecimal)
+
+  def * (that: Decimal): Decimal = Decimal(toBigDecimal * that.toBigDecimal)
+
+  def / (that: Decimal): Decimal =
+    if (that.isZero) null else Decimal(toBigDecimal / that.toBigDecimal)
+
+  def % (that: Decimal): Decimal =
+    if (that.isZero) null else Decimal(toBigDecimal % that.toBigDecimal)
+
+  def remainder(that: Decimal): Decimal = this % that
+
+  def unary_- : Decimal = {
+    if (decimalVal.ne(null)) {
+      Decimal(-decimalVal)
+    } else {
+      Decimal(-longVal, precision, scale)
+    }
+  }
+}
+
+object Decimal {
+  private val ROUNDING_MODE = BigDecimal.RoundingMode.HALF_UP
+
+  /** Maximum number of decimal digits a Long can represent */
+  val MAX_LONG_DIGITS = 18
+
+  private val POW_10 = Array.tabulate[Long](MAX_LONG_DIGITS + 1)(i => math.pow(10, i).toLong)
+
+  private val BIG_DEC_ZERO = BigDecimal(0)
+
+  def apply(value: Double): Decimal = new Decimal().set(value)
+
+  def apply(value: Long): Decimal = new Decimal().set(value)
+
+  def apply(value: Int): Decimal = new Decimal().set(value)
+
+  def apply(value: BigDecimal): Decimal = new Decimal().set(value)
+
+  def apply(value: BigDecimal, precision: Int, scale: Int): Decimal =
+    new Decimal().set(value, precision, scale)
+
+  def apply(unscaled: Long, precision: Int, scale: Int): Decimal =
+    new Decimal().set(unscaled, precision, scale)
+
+  def apply(value: String): Decimal = new Decimal().set(BigDecimal(value))
+
+  // Evidence parameters for Decimal considered either as Fractional or Integral. We provide two
+  // parameters inheriting from a common trait since both traits define mkNumericOps.
+  // See scala.math's Numeric.scala for examples for Scala's built-in types.
+
+  /** Common methods for Decimal evidence parameters */
+  trait DecimalIsConflicted extends Numeric[Decimal] {
+    override def plus(x: Decimal, y: Decimal): Decimal = x + y
+    override def times(x: Decimal, y: Decimal): Decimal = x * y
+    override def minus(x: Decimal, y: Decimal): Decimal = x - y
+    override def negate(x: Decimal): Decimal = -x
+    override def toDouble(x: Decimal): Double = x.toDouble
+    override def toFloat(x: Decimal): Float = x.toFloat
+    override def toInt(x: Decimal): Int = x.toInt
+    override def toLong(x: Decimal): Long = x.toLong
+    override def fromInt(x: Int): Decimal = new Decimal().set(x)
+    override def compare(x: Decimal, y: Decimal): Int = x.compare(y)
+  }
+
+  /** A [[scala.math.Fractional]] evidence parameter for Decimals. */
+  object DecimalIsFractional extends DecimalIsConflicted with Fractional[Decimal] {
+    override def div(x: Decimal, y: Decimal): Decimal = x / y
+  }
+
+  /** A [[scala.math.Integral]] evidence parameter for Decimals. */
+  object DecimalAsIfIntegral extends DecimalIsConflicted with Integral[Decimal] {
+    override def quot(x: Decimal, y: Decimal): Decimal = x / y
+    override def rem(x: Decimal, y: Decimal): Decimal = x % y
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/main/scala/org/apache/spark/sql/types/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/package.scala
new file mode 100644
index 0000000..346a51e
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/package.scala
@@ -0,0 +1,24 @@
+/*
+ * 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
+
+/**
+ * Contains a type system for attributes produced by relations, including complex types like
+ * structs, arrays and maps.
+ */
+package object types

http://git-wip-us.apache.org/repos/asf/spark/blob/f9969098/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
index 7be24be..117725d 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
@@ -23,7 +23,7 @@ import java.sql.{Date, Timestamp}
 import org.scalatest.FunSuite
 
 import org.apache.spark.sql.catalyst.expressions.Row
-import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.types._
 
 case class PrimitiveData(
     intField: Int,


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