aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala5
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/Transformer.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala1
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala3
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala1
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala3
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala3
-rw-r--r--project/MimaExcludes.scala12
-rw-r--r--project/SparkBuild.scala4
-rw-r--r--sql/README.md2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala5
-rwxr-xr-xsql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala2
-rwxr-xr-xsql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala3
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala3
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala2
-rwxr-xr-xsql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala10
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala3
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala17
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala68
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypes.java (renamed from sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java)50
-rwxr-xr-xsql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala (renamed from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala)16
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/types/SQLUserDefinedType.java (renamed from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java)3
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala (renamed from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala)294
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/types/decimal/Decimal.scala (renamed from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala)2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/types/package.scala (renamed from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala)3
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala2
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala2
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala2
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala2
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala10
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala2
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala2
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala2
-rwxr-xr-xsql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala2
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala (renamed from sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala)2
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala (renamed from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala)2
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java68
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java27
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java27
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java27
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java27
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java79
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java27
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java27
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java27
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java27
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java78
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java31
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java28
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java27
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java27
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java27
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java91
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java58
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java27
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java54
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala9
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala6
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala69
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala10
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala139
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala75
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala3
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala3
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala6
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala3
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala8
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/package.scala349
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala17
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala16
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala7
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala10
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala3
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala175
-rw-r--r--sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java9
-rw-r--r--sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java23
-rw-r--r--sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java150
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala1
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala1
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala3
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala1
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala3
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala9
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala89
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala16
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala6
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala16
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala21
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala1
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala2
-rw-r--r--sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala4
-rw-r--r--sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala4
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala2
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala15
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala2
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala5
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala7
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala2
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala4
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala4
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala2
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala19
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala2
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala1
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala1
-rw-r--r--sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala20
-rw-r--r--sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala11
160 files changed, 756 insertions, 2096 deletions
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 081a574bee..ad6fed178f 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 ::
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 23fbd228d0..1331b91240 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 ::
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 85b8899636..8c570812f8 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
/**
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 0b0504e036..12473cb2b5 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 ::
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 e0bfb1e484..0956062643 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 ::
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 896a6b83b6..72825f6e02 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]].
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 9352f40f37..e622a5cf9e 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 ::
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 194b9bfd9a..08fe991764 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]].
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 bf1faa25ef..adbd8266ed 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.
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 51e8bd4cf6..f6f9f491f4 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") =>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 46a54c6818..b2c546da21 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
diff --git a/sql/README.md b/sql/README.md
index 8d2f3cf428..d058a6b011 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.
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 2cf241de61..d169da691d 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
/**
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 5d974df98b..d19563e95c 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:
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 c009cc1e1e..bd00ff22ba 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
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 242f28f670..15353361d9 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.
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 b2262e5e6e..bdac7504ed 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.
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 fa80b07f8e..76a9f08dea 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
/**
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 4ede0b4821..00961f0991 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 {
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 ac5b02c2e6..cf14992ef8 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 =>
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 851db95b91..b2c6d30290 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 {
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 463f3667fc..dcda53bb71 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 {
/**
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 18c96da2f8..8a36c68107 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.
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 570379c533..37d9f0ed5c 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,
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 1a4ac06c7a..8328278544 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.
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 5ea9868e9e..735b7488fd 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
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 168a963e29..574907f566 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
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 90c81b2631..a5d6423391 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))
}
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 094ff14552..0db29eb404 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
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 2ff61169a1..cc97cb4f50 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._
/**
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 9aec601886..1bc34f7144 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`.
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 d1eab2eb4e..e54cfa144a 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 {
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 ab0701fd9a..43b6482c01 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.
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 94e1d37c1c..8ee4bbd8ca 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 {
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 a3c300b5d9..3035d934ff 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()
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 cb5ff67959..c84cc95520 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) =
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 3d4c4a8853..3a5bdca1f0 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
/**
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 f634976776..f85ee0a9bb 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 =>
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 cd3137980c..17b4f9c23a 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]
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 a38079ced3..105cdf5250 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
}
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 dcbbb62c0a..619f42859c 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 =>
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 ed578e081b..65ae066e4b 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")
}
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 0b9f01cbae..1483beacc9 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)
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 ccb0df113c..3c3d7a3119 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
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 0000000000..2a8914cde2
--- /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
+ }
+}
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypes.java
index c69bbd5736..e457542c64 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypes.java
@@ -15,77 +15,74 @@
* limitations under the License.
*/
-package org.apache.spark.sql.api.java;
+package org.apache.spark.sql.types;
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 {
-
+public class DataTypes {
/**
* Gets the StringType object.
*/
- public static final StringType StringType = new StringType();
+ public static final DataType StringType = StringType$.MODULE$;
/**
* Gets the BinaryType object.
*/
- public static final BinaryType BinaryType = new BinaryType();
+ public static final DataType BinaryType = BinaryType$.MODULE$;
/**
* Gets the BooleanType object.
*/
- public static final BooleanType BooleanType = new BooleanType();
+ public static final DataType BooleanType = BooleanType$.MODULE$;
/**
* Gets the DateType object.
*/
- public static final DateType DateType = new DateType();
+ public static final DataType DateType = DateType$.MODULE$;
/**
* Gets the TimestampType object.
*/
- public static final TimestampType TimestampType = new TimestampType();
+ public static final DataType TimestampType = TimestampType$.MODULE$;
/**
* Gets the DoubleType object.
*/
- public static final DoubleType DoubleType = new DoubleType();
+ public static final DataType DoubleType = DoubleType$.MODULE$;
/**
* Gets the FloatType object.
*/
- public static final FloatType FloatType = new FloatType();
+ public static final DataType FloatType = FloatType$.MODULE$;
/**
* Gets the ByteType object.
*/
- public static final ByteType ByteType = new ByteType();
+ public static final DataType ByteType = ByteType$.MODULE$;
/**
* Gets the IntegerType object.
*/
- public static final IntegerType IntegerType = new IntegerType();
+ public static final DataType IntegerType = IntegerType$.MODULE$;
/**
* Gets the LongType object.
*/
- public static final LongType LongType = new LongType();
+ public static final DataType LongType = LongType$.MODULE$;
/**
* Gets the ShortType object.
*/
- public static final ShortType ShortType = new ShortType();
+ public static final DataType ShortType = ShortType$.MODULE$;
/**
* Gets the NullType object.
*/
- public static final NullType NullType = new NullType();
+ public static final DataType NullType = NullType$.MODULE$;
/**
* Creates an ArrayType by specifying the data type of elements ({@code elementType}).
@@ -95,7 +92,6 @@ public abstract class DataType {
if (elementType == null) {
throw new IllegalArgumentException("elementType should not be null.");
}
-
return new ArrayType(elementType, true);
}
@@ -107,10 +103,17 @@ public abstract class DataType {
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}.
@@ -122,7 +125,6 @@ public abstract class DataType {
if (valueType == null) {
throw new IllegalArgumentException("valueType should not be null.");
}
-
return new MapType(keyType, valueType, true);
}
@@ -141,7 +143,6 @@ public abstract class DataType {
if (valueType == null) {
throw new IllegalArgumentException("valueType should not be null.");
}
-
return new MapType(keyType, valueType, valueContainsNull);
}
@@ -163,7 +164,6 @@ public abstract class DataType {
if (metadata == null) {
throw new IllegalArgumentException("metadata should not be null.");
}
-
return new StructField(name, dataType, nullable, metadata);
}
@@ -191,18 +191,18 @@ public abstract class DataType {
throw new IllegalArgumentException("fields should not be null.");
}
Set<String> distinctNames = new HashSet<String>();
- for (StructField field: fields) {
+ for (StructField field : fields) {
if (field == null) {
throw new IllegalArgumentException(
"fields should not contain any null.");
}
- distinctNames.add(field.getName());
+ distinctNames.add(field.name());
}
if (distinctNames.size() != fields.length) {
throw new IllegalArgumentException("fields should have distinct names.");
}
- return new StructType(fields);
+ return StructType$.MODULE$.apply(fields);
}
}
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/types/Metadata.scala
index 8172733e94..e50e976143 100755
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala
@@ -15,24 +15,31 @@
* limitations under the License.
*/
-package org.apache.spark.sql.catalyst.util
+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.
+ * [[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 {
+@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)
@@ -201,8 +208,11 @@ object Metadata {
}
/**
+ * :: 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
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/types/SQLUserDefinedType.java
index e966aeea1c..a64d2bb7cd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/SQLUserDefinedType.java
@@ -15,12 +15,11 @@
* limitations under the License.
*/
-package org.apache.spark.sql.catalyst.annotation;
+package org.apache.spark.sql.types;
import java.lang.annotation.*;
import org.apache.spark.annotation.DeveloperApi;
-import org.apache.spark.sql.catalyst.types.UserDefinedType;
/**
* ::DeveloperApi::
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/types/dataTypes.scala
index 892b7e1a97..fa0a355ebc 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala
@@ -15,11 +15,11 @@
* limitations under the License.
*/
-package org.apache.spark.sql.catalyst.types
+package org.apache.spark.sql.types
import java.sql.{Date, Timestamp}
-import scala.math.Numeric.{FloatAsIfIntegral, BigDecimalAsIfIntegral, DoubleAsIfIntegral}
+import scala.math.Numeric.{FloatAsIfIntegral, DoubleAsIfIntegral}
import scala.reflect.ClassTag
import scala.reflect.runtime.universe.{TypeTag, runtimeMirror, typeTag}
import scala.util.parsing.combinator.RegexParsers
@@ -31,11 +31,11 @@ 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.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))
@@ -140,7 +140,7 @@ object DataType {
protected lazy val structType: Parser[DataType] =
"StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ {
- case fields => new StructType(fields)
+ case fields => StructType(fields)
}
protected lazy val dataType: Parser[DataType] =
@@ -181,7 +181,7 @@ object DataType {
/**
* Compares two types, ignoring nullability of ArrayType, MapType, StructType.
*/
- def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = {
+ private[sql] def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = {
(left, right) match {
case (ArrayType(leftElementType, _), ArrayType(rightElementType, _)) =>
equalsIgnoreNullability(leftElementType, rightElementType)
@@ -200,6 +200,15 @@ object DataType {
}
}
+
+/**
+ * :: 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 {
@@ -218,8 +227,18 @@ abstract class DataType {
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)
@@ -237,10 +256,12 @@ object NativeType {
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
@@ -267,12 +288,31 @@ abstract class NativeType extends DataType {
}
}
+
+/**
+ * :: 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] }
@@ -287,12 +327,31 @@ case object BinaryType extends NativeType with PrimitiveType {
}
}
+
+/**
+ * :: 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
@@ -303,6 +362,16 @@ case object TimestampType extends NativeType {
}
}
+
+/**
+ * :: 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
@@ -313,6 +382,7 @@ case object DateType extends NativeType {
}
}
+
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
@@ -322,10 +392,12 @@ abstract class NumericType extends NativeType with PrimitiveType {
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 {
@@ -334,10 +406,20 @@ object IntegralType {
}
}
-abstract class IntegralType extends NumericType {
+
+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] }
@@ -346,6 +428,15 @@ case object LongType extends IntegralType {
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] }
@@ -354,6 +445,15 @@ case object IntegerType extends IntegralType {
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] }
@@ -362,6 +462,15 @@ case object ShortType extends IntegralType {
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] }
@@ -370,6 +479,7 @@ case object ByteType extends IntegralType {
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 {
@@ -378,15 +488,28 @@ object FractionalType {
}
}
-abstract class FractionalType extends NumericType {
+
+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)
-/** A Decimal that might have fixed precision and scale, or unlimited values for these */
+
+/**
+ * :: 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] }
@@ -395,6 +518,10 @@ case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalT
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"
@@ -406,6 +533,7 @@ case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalT
}
}
+
/** Extra factory methods and pattern matchers for Decimals */
object DecimalType {
val Unlimited: DecimalType = DecimalType(None)
@@ -437,6 +565,15 @@ object DecimalType {
}
}
+
+/**
+ * :: 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] }
@@ -446,6 +583,15 @@ case object DoubleType extends FractionalType {
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] }
@@ -455,18 +601,31 @@ case object FloatType extends FractionalType {
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(
@@ -480,8 +639,10 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT
("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.
@@ -510,19 +671,92 @@ case class StructField(
}
}
+
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]])
+ }
}
-case class StructType(fields: Seq[StructField]) extends DataType {
- /**
- * Returns all field names in a [[Seq]].
- */
- lazy val fieldNames: Seq[String] = fields.map(_.name)
+/**
+ * :: 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.
@@ -532,8 +766,8 @@ case class StructType(fields: Seq[StructField]) extends DataType {
}
/**
- * Returns a [[StructType]] containing [[StructField]]s of the given names.
- * Those names which do not have matching fields will be ignored.
+ * 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
@@ -545,8 +779,8 @@ case class StructType(fields: Seq[StructField]) extends DataType {
StructType(fields.filter(f => names.contains(f.name)))
}
- protected[sql] def toAttributes =
- fields.map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)())
+ protected[sql] def toAttributes: Seq[AttributeReference] =
+ map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)())
def treeString: String = {
val builder = new StringBuilder
@@ -565,23 +799,38 @@ case class StructType(fields: Seq[StructField]) extends DataType {
override private[sql] def jsonValue =
("type" -> typeName) ~
- ("fields" -> fields.map(_.jsonValue))
+ ("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, true)
+ 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,
@@ -602,6 +851,7 @@ case class MapType(
("valueContainsNull" -> valueContainsNull)
}
+
/**
* ::DeveloperApi::
* The data type for User Defined Types (UDTs).
@@ -611,7 +861,7 @@ case class MapType(
* 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]].
+ * [[SQLUserDefinedType]].
*
* The conversion via `serialize` occurs when instantiating a `SchemaRDD` from another RDD.
* The conversion via `deserialize` occurs when reading from a `SchemaRDD`.
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/types/decimal/Decimal.scala
index 708362acf3..c7864d1ae9 100644
--- 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/types/decimal/Decimal.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.catalyst.types.decimal
+package org.apache.spark.sql.types.decimal
import org.apache.spark.annotation.DeveloperApi
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/types/package.scala
index de24449590..346a51ea10 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/package.scala
@@ -15,7 +15,8 @@
* limitations under the License.
*/
-package org.apache.spark.sql.catalyst
+package org.apache.spark.sql
+
/**
* Contains a type system for attributes produced by relations, including complex types like
* structs, arrays and maps.
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 7be24bea7d..117725df32 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,
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 f430057ef7..3aea337460 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._
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 bbbeb4f2e4..bc2ec754d5 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 {
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 dfa2d958c0..f5a502b43f 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 {
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 4ba7d87ba8..8552448b8d 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 {
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 0a27cce337..9fdf3efa02 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._
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 017b180c57..da912ab382 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._
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 036fd3fa1d..cdb843f959 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
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 f005b7df21..d7d60efee5 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()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala
index e9740d913c..892195f46e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
+package org.apache.spark.sql.types
import org.scalatest.FunSuite
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/types/decimal/DecimalSuite.scala
index e32f1ac382..813377df00 100644
--- 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/types/decimal/DecimalSuite.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.catalyst.types.decimal
+package org.apache.spark.sql.types.decimal
import org.scalatest.{PrivateMethodTester, FunSuite}
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 b73a371e93..0000000000
--- 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;
- }
-}
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 7daad60f62..0000000000
--- 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() {}
-}
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 5a1f527256..0000000000
--- 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() {}
-}
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 e5cdf06b21..0000000000
--- 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() {}
-}
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 6677793baa..0000000000
--- 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() {}
-}
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 60752451ec..0000000000
--- 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;
- }
-}
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 f0060d0bcf..0000000000
--- 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() {}
-}
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 4a6a37f691..0000000000
--- 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() {}
-}
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 bfd70490bb..0000000000
--- 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() {}
-}
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 af13a46eb1..0000000000
--- 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() {}
-}
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 063e6b34ab..0000000000
--- 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;
- }
-}
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 0f819fb01a..0000000000
--- 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);
- }
-}
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 6e6b12f072..0000000000
--- 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());
- }
-}
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 6d5ecdf46e..0000000000
--- 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() {}
-}
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 7d7604b4e3..0000000000
--- 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() {}
-}
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 f4ba0c07c9..0000000000
--- 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() {}
-}
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 7c60d492bc..0000000000
--- 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;
- }
-}
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 a4b501efd9..0000000000
--- 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);
- }
-}
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 06d44c731c..0000000000
--- 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() {}
-}
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 f0d079d25b..0000000000
--- 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();
-}
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 d8efce0cb4..d9f3b3a53f 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 ::
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 80787b61ce..686bcdfbb4 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
/**
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 65358b7d4e..f10ee7b66f 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
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 7f868cd4af..a75f559928 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)()
}
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 5b9c612487..9e10e532fb 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
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 401798e317..207e2805ff 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
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 158f26e3d4..4186c27451 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
}
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 a7d0f4f127..0000000000
--- 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)
- }
- }
-}
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 538dd5b734..91c4c105b1 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
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 c68dceef3b..3a4977b836 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 {
/**
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 668efe4a3b..391b3dae5c 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)()
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 ab66c85c4f..fcf2faa091 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
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 27ac5f4dbd..7dff9deac8 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] =>
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 628d9cec41..aead768ecd 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
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 acb06cb537..879d29bcfa 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 = {}
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 29edcf1724..6467324839 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
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 069e950195..20b14834bb 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 ::
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 7c3bf947e7..4abe26fe4a 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],
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 84d96e612f..131146012e 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
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 0652d2ff7c..0cc9d049c9 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] {
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 61be5ed2db..46245cd5a1 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 ::
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 5a41399971..741ccb8fb8 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}
/**
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 f5c02224c8..1af96c28d5 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 {
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 00449c2007..c92ec543e2 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)
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 1fd8e6220f..b75266d5aa 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
}
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 1bbb66aaa1..7f437c4077 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
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 56e7d11b2f..f08350878f 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"
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 9049eb5932..af7248fdf4 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.
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 97447871a1..6d8c682ccc 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 =
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 55a2728a85..1b50afbbab 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
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 4d87f6817d..12b59ba20b 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 {
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 f8741e0082..4cc9641c4d 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] =
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 7f5564baa0..cd82cc6ecb 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::
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 b9569e96c0..006b16fbe0 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.
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 d4ef517981..0000000000
--- 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
- }
-}
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 a9a11285de..88017eb47d 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();
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 a04b8060cd..de586ba635 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(
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 8396a29c61..0000000000
--- 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) {
- }
- }
-}
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 ab88f3ad10..efe622f8bc 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._
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 811319e0a6..f5b945f468 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 {
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 bc72daf088..cbdb3e64bb 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 = {
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 cf3a59e545..40fb8d5779 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._
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 1806a1dd82..a0d54d17f5 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 {
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 8afc3a9fb2..fdbb4282ba 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") {
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 62fe59dd34..0000000000
--- 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)
- }
-}
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 a9f0851f88..9be0b38e68 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))
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 3f3f35d501..87e608a885 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
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 a1f21219ea..f941465fa3 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) = {
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 21906e3fdc..f95c895587 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,
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 cb73f3da81..80bd5c9457 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)
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 1cdb909146..c82d979935 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)
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 73f31c0233..88011631ee 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)
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 4ce2552112..08df1db375 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)
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 7db723d648..0b18b41192 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,
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 b09f1ac495..01c1ce2a61 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") {
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 6ac67fcafe..973819aaa4 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
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 0a92336a3c..fe781ec05f 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
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 34d61bf908..64274950b8 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)
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 939b3c0c66..390538d35a 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(
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 fee2e22611..7900b3e894 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(
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 a1d2468b25..382dddcdea 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
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 742acba58d..171d707b13 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.
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 b82156427a..bec9d9aca3 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.
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 9aeebd7e54..bf56e60cf9 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.
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 a156d6f7e2..245b847cf4 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
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 785a6a14f4..d40f9936fd 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 */
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 34622b5f57..b13ef7276b 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._
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 cdff82e3d0..6952b126cf 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.
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 8ba818af5f..781a2e9164 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 ::
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 8bbcd6fec1..b56175fe76 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 ::
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 e70cdeaad4..cf72345efa 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 ::
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 7d863f9d89..d898b876c3 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 */
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 f90d360791..dc23d9a101 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 = {
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 041a36f129..fa6905f31f 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 {
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 fb481edc85..7cfb875e05 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._
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 ec9ebb4a77..8ff833e0d6 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._
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 25fdf5c5f3..a5587460fd 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
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 e47002cb0b..a7121360dd 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
/**