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 081a574beea5d..ad6fed178fae9 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 23fbd228d01cb..1331b9124045c 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 85b8899636ca5..8c570812f8316 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 0b0504e036ec9..12473cb2b5719 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 e0bfb1e484a2e..0956062643f23 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 896a6b83b67bf..72825f6e02182 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 9352f40f372d3..e622a5cf9e6f3 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 194b9bfd9a9e6..08fe99176424a 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 d40f13342a3d9..dc40ba7fc8d4b 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 31d4c317ae569..06b503e55920c 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -61,6 +61,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 46a54c6818409..b2c546da21c70 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 8d2f3cf4283e0..d058a6b011d37 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 2cf241de61f7a..d169da691d797 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 f79d4ff444dc0..3d47d145bfa94 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 72680f37a0b4d..c533c0ab07f5c 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 242f28f670298..15353361d97cb 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 9608e15c0f302..2ffe5664d7da9 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 fa80b07f8e6be..76a9f08dea85f 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 4ede0b4821fe3..00961f09916b3 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 ac5b02c2e6aed..cf14992ef835c 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 851db95b9177e..b2c6d3029031d 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 463f3667fc445..dcda53bb717a9 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 18c96da2f87fb..8a36c6810790d 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 570379c533e1f..37d9f0ed5c79e 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 1a4ac06c7a79d..8328278544a1e 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 5ea9868e9e846..735b7488fdcbd 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 168a963e29c90..574907f566c0f 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 90c81b2631e59..a5d6423391295 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 094ff14552283..0db29eb404bd1 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 2ff61169a17db..cc97cb4f50b69 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 9aec601886efc..1bc34f71441fe 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 d1eab2eb4ed56..e54cfa144a173 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 ab0701fd9a80b..43b6482c0171c 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 94e1d37c1c3ab..8ee4bbd8caa62 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 a3c300b5d90e9..3035d934ff9f8 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 cb5ff67959868..c84cc95520a19 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 3d4c4a8853c12..3a5bdca1f07c3 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 f6349767764a3..f85ee0a9bb6d8 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 cd3137980ca43..17b4f9c23a978 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 a38079ced34b2..105cdf52500c6 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 dcbbb62c0aca4..619f42859cbb8 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 ed578e081be73..65ae066e4b4b5 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 0b9f01cbae9ea..1483beacc9088 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/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 5a1863953eae9..37d643ea09e7a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.types.StringType +import org.apache.spark.sql.types.StringType /** * A logical node that represents a non-query command to be executed by the system. For example, 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 ccb0df113c063..3c3d7a3119064 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/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index 892b7e1a97c8b..2f3c96a004c73 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/catalyst/types/dataTypes.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.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 @@ -29,13 +29,20 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.ScalaReflectionLock -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, Row} -import org.apache.spark.sql.catalyst.types.decimal._ -import org.apache.spark.sql.catalyst.util.Metadata +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.types.Metadata +import org.apache.spark.sql.types.decimal.Decimal import org.apache.spark.util.Utils +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") object DataType { def fromJson(json: String): DataType = parseDataType(parse(json)) @@ -68,13 +75,6 @@ object DataType { ("fields", JArray(fields)), ("type", JString("struct"))) => StructType(fields.map(parseStructField)) - - case JSortedObject( - ("class", JString(udtClass)), - ("pyClass", _), - ("sqlType", _), - ("type", JString("udt"))) => - Class.forName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]] } private def parseStructField(json: JValue): StructField = json match { @@ -200,6 +200,15 @@ object DataType { } } + +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") abstract class DataType { /** Matches any expression that evaluates to this DataType */ def unapply(a: Expression): Boolean = a match { @@ -216,10 +225,30 @@ abstract class DataType { def json: String = compact(render(jsonValue)) def prettyJson: String = pretty(render(jsonValue)) + + def toNewType: org.apache.spark.sql.types.DataType } -case object NullType extends DataType +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") +case object NullType extends DataType { + override def toNewType = org.apache.spark.sql.types.NullType +} +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") object NativeType { val all = Seq( IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) @@ -237,10 +266,26 @@ object NativeType { StringType -> 4096) } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") trait PrimitiveType extends DataType { override def isPrimitive = true } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") object PrimitiveType { private val nonDecimals = Seq(NullType, DateType, TimestampType, BinaryType) ++ NativeType.all private val nonDecimalNameToType = nonDecimals.map(t => t.typeName -> t).toMap @@ -256,6 +301,14 @@ object PrimitiveType { } } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") abstract class NativeType extends DataType { private[sql] type JvmType @transient private[sql] val tag: TypeTag[JvmType] @@ -267,12 +320,29 @@ abstract class NativeType extends DataType { } } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") 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]] + override def toNewType = org.apache.spark.sql.types.StringType } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case object BinaryType extends NativeType with PrimitiveType { private[sql] type JvmType = Array[Byte] @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -285,14 +355,32 @@ case object BinaryType extends NativeType with PrimitiveType { x.length - y.length } } + override def toNewType = org.apache.spark.sql.types.BinaryType } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") 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]] + override def toNewType = org.apache.spark.sql.types.BooleanType } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case object TimestampType extends NativeType { private[sql] type JvmType = Timestamp @@ -301,8 +389,18 @@ case object TimestampType extends NativeType { private[sql] val ordering = new Ordering[JvmType] { def compare(x: Timestamp, y: Timestamp) = x.compareTo(y) } + + override def toNewType = org.apache.spark.sql.types.TimestampType } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case object DateType extends NativeType { private[sql] type JvmType = Date @@ -311,8 +409,18 @@ case object DateType extends NativeType { private[sql] val ordering = new Ordering[JvmType] { def compare(x: Date, y: Date) = x.compareTo(y) } + + override def toNewType = org.apache.spark.sql.types.DateType } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") 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,71 +430,119 @@ 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 { - case e: Expression if e.dataType.isInstanceOf[IntegralType] => true - case _ => false - } -} - +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") abstract class IntegralType extends NumericType { private[sql] val integral: Integral[JvmType] } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case object LongType extends IntegralType { private[sql] type JvmType = Long @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val numeric = implicitly[Numeric[Long]] private[sql] val integral = implicitly[Integral[Long]] private[sql] val ordering = implicitly[Ordering[JvmType]] + override def toNewType = org.apache.spark.sql.types.LongType } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case object IntegerType extends IntegralType { private[sql] type JvmType = Int @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val numeric = implicitly[Numeric[Int]] private[sql] val integral = implicitly[Integral[Int]] private[sql] val ordering = implicitly[Ordering[JvmType]] + override def toNewType = org.apache.spark.sql.types.IntegerType } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case object ShortType extends IntegralType { private[sql] type JvmType = Short @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val numeric = implicitly[Numeric[Short]] private[sql] val integral = implicitly[Integral[Short]] private[sql] val ordering = implicitly[Ordering[JvmType]] + override def toNewType = org.apache.spark.sql.types.ShortType } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case object ByteType extends IntegralType { private[sql] type JvmType = Byte @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val numeric = implicitly[Numeric[Byte]] private[sql] val integral = implicitly[Integral[Byte]] private[sql] val ordering = implicitly[Ordering[JvmType]] + override def toNewType = org.apache.spark.sql.types.ByteType } -/** Matcher for any expressions that evaluate to [[FractionalType]]s */ -object FractionalType { - def unapply(a: Expression): Boolean = a match { - case e: Expression if e.dataType.isInstanceOf[FractionalType] => true - case _ => false - } -} - +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") 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) +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") +case class PrecisionInfo(precision: Int, scale: Int) { + def toNewType = org.apache.spark.sql.types.PrecisionInfo(precision, scale) +} -/** A Decimal that might have fixed precision and scale, or unlimited values for these */ +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalType { private[sql] type JvmType = Decimal @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -404,9 +560,18 @@ case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalT case Some(PrecisionInfo(precision, scale)) => s"DecimalType($precision,$scale)" case None => "DecimalType()" } + + override def toNewType = org.apache.spark.sql.types.DecimalType(precisionInfo.map(_.toNewType)) } -/** Extra factory methods and pattern matchers for Decimals */ +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") object DecimalType { val Unlimited: DecimalType = DecimalType(None) @@ -415,28 +580,25 @@ object DecimalType { t.precisionInfo.map(p => (p.precision, p.scale)) } - object Expression { - def unapply(e: Expression): Option[(Int, Int)] = e.dataType match { - case t: DecimalType => t.precisionInfo.map(p => (p.precision, p.scale)) - case _ => None - } - } - def apply(): DecimalType = Unlimited def apply(precision: Int, scale: Int): DecimalType = DecimalType(Some(PrecisionInfo(precision, scale))) - def unapply(t: DataType): Boolean = t.isInstanceOf[DecimalType] - - def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[DecimalType] - def isFixed(dataType: DataType): Boolean = dataType match { case DecimalType.Fixed(_, _) => true case _ => false } } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case object DoubleType extends FractionalType { private[sql] type JvmType = Double @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -444,8 +606,17 @@ case object DoubleType extends FractionalType { private[sql] val fractional = implicitly[Fractional[Double]] private[sql] val ordering = implicitly[Ordering[JvmType]] private[sql] val asIntegral = DoubleAsIfIntegral + override def toNewType = org.apache.spark.sql.types.DoubleType } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case object FloatType extends FractionalType { private[sql] type JvmType = Float @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -453,20 +624,30 @@ case object FloatType extends FractionalType { private[sql] val fractional = implicitly[Fractional[Float]] private[sql] val ordering = implicitly[Ordering[JvmType]] private[sql] val asIntegral = FloatAsIfIntegral + override def toNewType = org.apache.spark.sql.types.FloatType } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") object ArrayType { /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ def apply(elementType: DataType): ArrayType = ArrayType(elementType, true) } /** - * The data type for collections of multiple values. - * Internally these are represented as columns that contain a ``scala.collection.Seq``. + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. * - * @param elementType The data type of values. - * @param containsNull Indicates if values have `null` values + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { builder.append( @@ -478,16 +659,18 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT ("type" -> typeName) ~ ("elementType" -> elementType.jsonValue) ~ ("containsNull" -> containsNull) + + override def toNewType = org.apache.spark.sql.types.ArrayType(elementType.toNewType, containsNull) } /** - * A field inside a StructType. - * @param name The name of this field. - * @param dataType The data type of this field. - * @param nullable Indicates if values of this field can be `null` values. - * @param metadata The metadata of this field. The metadata should be preserved during - * transformation if the content of the column is not modified, e.g, in selection. + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case class StructField( name: String, dataType: DataType, @@ -508,13 +691,19 @@ case class StructField( ("nullable" -> nullable) ~ ("metadata" -> metadata.jsonValue) } -} -object StructType { - protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = - StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) + def toNewType = org.apache.spark.sql.types.StructField( + name, dataType.toNewType, nullable, metadata) } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case class StructType(fields: Seq[StructField]) extends DataType { /** @@ -545,9 +734,6 @@ 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)()) - def treeString: String = { val builder = new StringBuilder builder.append("root\n") @@ -566,8 +752,18 @@ case class StructType(fields: Seq[StructField]) extends DataType { override private[sql] def jsonValue = ("type" -> typeName) ~ ("fields" -> fields.map(_.jsonValue)) + + override def toNewType = org.apache.spark.sql.types.StructType(fields.map(_.toNewType)) } +/** + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. + */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") object MapType { /** * Construct a [[MapType]] object with the given key type and value type. @@ -578,11 +774,13 @@ object MapType { } /** - * The data type for Maps. Keys in a map are not allowed to have `null` values. - * @param keyType The data type of map keys. - * @param valueType The data type of map values. - * @param valueContainsNull Indicates if map values have `null` values. + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. */ +@deprecated("1.3.0", "Use org.apache.spark.sql.types package instead.") case class MapType( keyType: DataType, valueType: DataType, @@ -600,51 +798,7 @@ case class MapType( ("keyType" -> keyType.jsonValue) ~ ("valueType" -> valueType.jsonValue) ~ ("valueContainsNull" -> valueContainsNull) -} -/** - * ::DeveloperApi:: - * The data type for User Defined Types (UDTs). - * - * This interface allows a user to make their own classes more interoperable with SparkSQL; - * e.g., by creating a [[UserDefinedType]] for a class X, it becomes possible to create - * a SchemaRDD which has class X in the schema. - * - * For SparkSQL to recognize UDTs, the UDT must be annotated with - * [[org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType]]. - * - * The conversion via `serialize` occurs when instantiating a `SchemaRDD` from another RDD. - * The conversion via `deserialize` occurs when reading from a `SchemaRDD`. - */ -@DeveloperApi -abstract class UserDefinedType[UserType] extends DataType with Serializable { - - /** Underlying storage type for this UDT */ - def sqlType: DataType - - /** Paired Python UDT class, if exists. */ - def pyUDT: String = null - - /** - * Convert the user type to a SQL datum - * - * TODO: Can we make this take obj: UserType? The issue is in ScalaReflection.convertToCatalyst, - * where we need to convert Any to UserType. - */ - def serialize(obj: Any): Any - - /** Convert a SQL datum to the user type */ - def deserialize(datum: Any): UserType - - override private[sql] def jsonValue: JValue = { - ("type" -> "udt") ~ - ("class" -> this.getClass.getName) ~ - ("pyClass" -> pyUDT) ~ - ("sqlType" -> sqlType.jsonValue) - } - - /** - * Class object for the UserType - */ - def userClass: java.lang.Class[UserType] + override def toNewType = org.apache.spark.sql.types.MapType( + keyType.toNewType, valueType.toNewType, valueContainsNull) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala index de24449590f9a..8df2f7e1df1f7 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/catalyst/types/package.scala @@ -16,8 +16,12 @@ */ package org.apache.spark.sql.catalyst + /** - * Contains a type system for attributes produced by relations, including complex types like - * structs, arrays and maps. + * This package is deprecated in Spark 1.3.0 and is only kept here for maintaining + * backward compatibility for external data sources written against Spark 1.2.0. + * + * The entire package will be removed in future versions of Spark (likely 1.4.0). + * Please see https://issues.apache.org/jira/browse/SPARK-5123 for more information. */ package object types 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 0000000000000..2a8914cde248c --- /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 similarity index 81% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypes.java index c69bbd5736a5b..e457542c647e7 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 static ArrayType createArrayType(DataType elementType) { if (elementType == null) { throw new IllegalArgumentException("elementType should not be null."); } - return new ArrayType(elementType, true); } @@ -107,10 +103,17 @@ public static ArrayType createArrayType(DataType elementType, boolean containsNu 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 static MapType createMapType(DataType keyType, DataType valueType) { if (valueType == null) { throw new IllegalArgumentException("valueType should not be null."); } - return new MapType(keyType, valueType, true); } @@ -141,7 +143,6 @@ public static MapType createMapType( if (valueType == null) { throw new IllegalArgumentException("valueType should not be null."); } - return new MapType(keyType, valueType, valueContainsNull); } @@ -163,7 +164,6 @@ public static StructField createStructField( if (metadata == null) { throw new IllegalArgumentException("metadata should not be null."); } - return new StructField(name, dataType, nullable, metadata); } @@ -191,18 +191,18 @@ public static StructType createStructType(StructField[] fields) { throw new IllegalArgumentException("fields should not be null."); } Set distinctNames = new HashSet(); - 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 similarity index 96% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala index 8172733e94dd5..e50e9761431f5 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 similarity index 93% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/SQLUserDefinedType.java index e966aeea1cb23..a64d2bb7cde37 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/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala new file mode 100644 index 0000000000000..fa0a355ebc9b0 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -0,0 +1,900 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.types + +import java.sql.{Date, Timestamp} + +import scala.math.Numeric.{FloatAsIfIntegral, DoubleAsIfIntegral} +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.{TypeTag, runtimeMirror, typeTag} +import scala.util.parsing.combinator.RegexParsers + +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} +import org.apache.spark.sql.types.decimal._ +import org.apache.spark.util.Utils + + +object DataType { + def fromJson(json: String): DataType = parseDataType(parse(json)) + + private object JSortedObject { + def unapplySeq(value: JValue): Option[List[(String, JValue)]] = value match { + case JObject(seq) => Some(seq.toList.sortBy(_._1)) + case _ => None + } + } + + // NOTE: Map fields must be sorted in alphabetical order to keep consistent with the Python side. + private def parseDataType(json: JValue): DataType = json match { + case JString(name) => + PrimitiveType.nameToType(name) + + case JSortedObject( + ("containsNull", JBool(n)), + ("elementType", t: JValue), + ("type", JString("array"))) => + ArrayType(parseDataType(t), n) + + case JSortedObject( + ("keyType", k: JValue), + ("type", JString("map")), + ("valueContainsNull", JBool(n)), + ("valueType", v: JValue)) => + MapType(parseDataType(k), parseDataType(v), n) + + case JSortedObject( + ("fields", JArray(fields)), + ("type", JString("struct"))) => + StructType(fields.map(parseStructField)) + + case JSortedObject( + ("class", JString(udtClass)), + ("pyClass", _), + ("sqlType", _), + ("type", JString("udt"))) => + Class.forName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]] + } + + private def parseStructField(json: JValue): StructField = json match { + case JSortedObject( + ("metadata", metadata: JObject), + ("name", JString(name)), + ("nullable", JBool(nullable)), + ("type", dataType: JValue)) => + StructField(name, parseDataType(dataType), nullable, Metadata.fromJObject(metadata)) + // Support reading schema when 'metadata' is missing. + case JSortedObject( + ("name", JString(name)), + ("nullable", JBool(nullable)), + ("type", dataType: JValue)) => + StructField(name, parseDataType(dataType), nullable) + } + + @deprecated("Use DataType.fromJson instead", "1.2.0") + def fromCaseClassString(string: String): DataType = CaseClassStringParser(string) + + private object CaseClassStringParser extends RegexParsers { + protected lazy val primitiveType: Parser[DataType] = + ( "StringType" ^^^ StringType + | "FloatType" ^^^ FloatType + | "IntegerType" ^^^ IntegerType + | "ByteType" ^^^ ByteType + | "ShortType" ^^^ ShortType + | "DoubleType" ^^^ DoubleType + | "LongType" ^^^ LongType + | "BinaryType" ^^^ BinaryType + | "BooleanType" ^^^ BooleanType + | "DateType" ^^^ DateType + | "DecimalType()" ^^^ DecimalType.Unlimited + | fixedDecimalType + | "TimestampType" ^^^ TimestampType + ) + + protected lazy val fixedDecimalType: Parser[DataType] = + ("DecimalType(" ~> "[0-9]+".r) ~ ("," ~> "[0-9]+".r <~ ")") ^^ { + case precision ~ scale => DecimalType(precision.toInt, scale.toInt) + } + + protected lazy val arrayType: Parser[DataType] = + "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ { + case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull) + } + + protected lazy val mapType: Parser[DataType] = + "MapType" ~> "(" ~> dataType ~ "," ~ dataType ~ "," ~ boolVal <~ ")" ^^ { + case t1 ~ _ ~ t2 ~ _ ~ valueContainsNull => MapType(t1, t2, valueContainsNull) + } + + protected lazy val structField: Parser[StructField] = + ("StructField(" ~> "[a-zA-Z0-9_]*".r) ~ ("," ~> dataType) ~ ("," ~> boolVal <~ ")") ^^ { + case name ~ tpe ~ nullable => + StructField(name, tpe, nullable = nullable) + } + + protected lazy val boolVal: Parser[Boolean] = + ( "true" ^^^ true + | "false" ^^^ false + ) + + protected lazy val structType: Parser[DataType] = + "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { + case fields => StructType(fields) + } + + protected lazy val dataType: Parser[DataType] = + ( arrayType + | mapType + | structType + | primitiveType + ) + + /** + * Parses a string representation of a DataType. + * + * TODO: Generate parser as pickler... + */ + def apply(asString: String): DataType = parseAll(dataType, asString) match { + case Success(result, _) => result + case failure: NoSuccess => + throw new IllegalArgumentException(s"Unsupported dataType: $asString, $failure") + } + + } + + protected[types] def buildFormattedString( + dataType: DataType, + prefix: String, + builder: StringBuilder): Unit = { + dataType match { + case array: ArrayType => + array.buildFormattedString(prefix, builder) + case struct: StructType => + struct.buildFormattedString(prefix, builder) + case map: MapType => + map.buildFormattedString(prefix, builder) + case _ => + } + } + + /** + * Compares two types, ignoring nullability of ArrayType, MapType, StructType. + */ + private[sql] def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = { + (left, right) match { + case (ArrayType(leftElementType, _), ArrayType(rightElementType, _)) => + equalsIgnoreNullability(leftElementType, rightElementType) + case (MapType(leftKeyType, leftValueType, _), MapType(rightKeyType, rightValueType, _)) => + equalsIgnoreNullability(leftKeyType, rightKeyType) && + equalsIgnoreNullability(leftValueType, rightValueType) + case (StructType(leftFields), StructType(rightFields)) => + leftFields.size == rightFields.size && + leftFields.zip(rightFields) + .forall{ + case (left, right) => + left.name == right.name && equalsIgnoreNullability(left.dataType, right.dataType) + } + case (left, right) => left == right + } + } +} + + +/** + * :: DeveloperApi :: + * + * The base type of all Spark SQL data types. + * + * @group dataType + */ +@DeveloperApi +abstract class DataType { + /** Matches any expression that evaluates to this DataType */ + def unapply(a: Expression): Boolean = a match { + case e: Expression if e.dataType == this => true + case _ => false + } + + def isPrimitive: Boolean = false + + def typeName: String = this.getClass.getSimpleName.stripSuffix("$").dropRight(4).toLowerCase + + private[sql] def jsonValue: JValue = typeName + + def json: String = compact(render(jsonValue)) + + def prettyJson: String = pretty(render(jsonValue)) +} + + +/** + * :: DeveloperApi :: + * + * The data type representing `NULL` values. Please use the singleton [[DataTypes.NullType]]. + * + * @group dataType + */ +@DeveloperApi +case object NullType extends DataType + + +object NativeType { + val all = Seq( + IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) + + def unapply(dt: DataType): Boolean = all.contains(dt) + + val defaultSizeOf: Map[NativeType, Int] = Map( + IntegerType -> 4, + BooleanType -> 1, + LongType -> 8, + DoubleType -> 8, + FloatType -> 4, + ShortType -> 2, + ByteType -> 1, + StringType -> 4096) +} + + +trait PrimitiveType extends DataType { + override def isPrimitive = true +} + + +object PrimitiveType { + private val nonDecimals = Seq(NullType, DateType, TimestampType, BinaryType) ++ NativeType.all + private val nonDecimalNameToType = nonDecimals.map(t => t.typeName -> t).toMap + + /** Given the string representation of a type, return its DataType */ + private[sql] def nameToType(name: String): DataType = { + val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\d+)\s*\)""".r + name match { + case "decimal" => DecimalType.Unlimited + case FIXED_DECIMAL(precision, scale) => DecimalType(precision.toInt, scale.toInt) + case other => nonDecimalNameToType(other) + } + } +} + +abstract class NativeType extends DataType { + private[sql] type JvmType + @transient private[sql] val tag: TypeTag[JvmType] + private[sql] val ordering: Ordering[JvmType] + + @transient private[sql] val classTag = ScalaReflectionLock.synchronized { + val mirror = runtimeMirror(Utils.getSparkClassLoader) + ClassTag[JvmType](mirror.runtimeClass(tag.tpe)) + } +} + + +/** + * :: DeveloperApi :: + * + * The data type representing `String` values. Please use the singleton [[DataTypes.StringType]]. + * + * @group dataType + */ +@DeveloperApi +case object StringType extends NativeType with PrimitiveType { + private[sql] type JvmType = String + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val ordering = implicitly[Ordering[JvmType]] +} + + +/** + * :: DeveloperApi :: + * + * The data type representing `Array[Byte]` values. + * Please use the singleton [[DataTypes.BinaryType]]. + * + * @group dataType + */ +@DeveloperApi +case object BinaryType extends NativeType with PrimitiveType { + private[sql] type JvmType = Array[Byte] + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val ordering = new Ordering[JvmType] { + def compare(x: Array[Byte], y: Array[Byte]): Int = { + for (i <- 0 until x.length; if i < y.length) { + val res = x(i).compareTo(y(i)) + if (res != 0) return res + } + x.length - y.length + } + } +} + + +/** + * :: DeveloperApi :: + * + * The data type representing `Boolean` values. Please use the singleton [[DataTypes.BooleanType]]. + * + *@group dataType + */ +@DeveloperApi +case object BooleanType extends NativeType with PrimitiveType { + private[sql] type JvmType = Boolean + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val ordering = implicitly[Ordering[JvmType]] +} + + +/** + * :: DeveloperApi :: + * + * The data type representing `java.sql.Timestamp` values. + * Please use the singleton [[DataTypes.TimestampType]]. + * + * @group dataType + */ +@DeveloperApi +case object TimestampType extends NativeType { + private[sql] type JvmType = Timestamp + + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + + private[sql] val ordering = new Ordering[JvmType] { + def compare(x: Timestamp, y: Timestamp) = x.compareTo(y) + } +} + + +/** + * :: DeveloperApi :: + * + * The data type representing `java.sql.Date` values. + * Please use the singleton [[DataTypes.DateType]]. + * + * @group dataType + */ +@DeveloperApi +case object DateType extends NativeType { + private[sql] type JvmType = Date + + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + + private[sql] val ordering = new Ordering[JvmType] { + def compare(x: Date, y: Date) = x.compareTo(y) + } +} + + +abstract class NumericType extends NativeType with PrimitiveType { + // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for + // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a + // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets + // desugared by the compiler into an argument to the objects constructor. This means there is no + // longer an no argument constructor and thus the JVM cannot serialize the object anymore. + private[sql] val numeric: Numeric[JvmType] +} + + +object NumericType { + def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[NumericType] +} + + +/** Matcher for any expressions that evaluate to [[IntegralType]]s */ +object IntegralType { + def unapply(a: Expression): Boolean = a match { + case e: Expression if e.dataType.isInstanceOf[IntegralType] => true + case _ => false + } +} + + +sealed abstract class IntegralType extends NumericType { + private[sql] val integral: Integral[JvmType] +} + + +/** + * :: DeveloperApi :: + * + * The data type representing `Long` values. Please use the singleton [[DataTypes.LongType]]. + * + * @group dataType + */ +@DeveloperApi +case object LongType extends IntegralType { + private[sql] type JvmType = Long + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Long]] + private[sql] val integral = implicitly[Integral[Long]] + private[sql] val ordering = implicitly[Ordering[JvmType]] +} + + +/** + * :: DeveloperApi :: + * + * The data type representing `Int` values. Please use the singleton [[DataTypes.IntegerType]]. + * + * @group dataType + */ +@DeveloperApi +case object IntegerType extends IntegralType { + private[sql] type JvmType = Int + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Int]] + private[sql] val integral = implicitly[Integral[Int]] + private[sql] val ordering = implicitly[Ordering[JvmType]] +} + + +/** + * :: DeveloperApi :: + * + * The data type representing `Short` values. Please use the singleton [[DataTypes.ShortType]]. + * + * @group dataType + */ +@DeveloperApi +case object ShortType extends IntegralType { + private[sql] type JvmType = Short + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Short]] + private[sql] val integral = implicitly[Integral[Short]] + private[sql] val ordering = implicitly[Ordering[JvmType]] +} + + +/** + * :: DeveloperApi :: + * + * The data type representing `Byte` values. Please use the singleton [[DataTypes.ByteType]]. + * + * @group dataType + */ +@DeveloperApi +case object ByteType extends IntegralType { + private[sql] type JvmType = Byte + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Byte]] + private[sql] val integral = implicitly[Integral[Byte]] + private[sql] val ordering = implicitly[Ordering[JvmType]] +} + + +/** Matcher for any expressions that evaluate to [[FractionalType]]s */ +object FractionalType { + def unapply(a: Expression): Boolean = a match { + case e: Expression if e.dataType.isInstanceOf[FractionalType] => true + case _ => false + } +} + + +sealed abstract class FractionalType extends NumericType { + private[sql] val fractional: Fractional[JvmType] + private[sql] val asIntegral: Integral[JvmType] +} + + +/** Precision parameters for a Decimal */ +case class PrecisionInfo(precision: Int, scale: Int) + + +/** + * :: DeveloperApi :: + * + * The data type representing `scala.math.BigDecimal` values. + * A Decimal that might have fixed precision and scale, or unlimited values for these. + * + * Please use [[DataTypes.createDecimalType()]] to create a specific instance. + * + * @group dataType + */ +@DeveloperApi +case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalType { + private[sql] type JvmType = Decimal + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = Decimal.DecimalIsFractional + private[sql] val fractional = Decimal.DecimalIsFractional + private[sql] val ordering = Decimal.DecimalIsFractional + private[sql] val asIntegral = Decimal.DecimalAsIfIntegral + + def precision: Int = precisionInfo.map(_.precision).getOrElse(-1) + + def scale: Int = precisionInfo.map(_.scale).getOrElse(-1) + + override def typeName: String = precisionInfo match { + case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" + case None => "decimal" + } + + override def toString: String = precisionInfo match { + case Some(PrecisionInfo(precision, scale)) => s"DecimalType($precision,$scale)" + case None => "DecimalType()" + } +} + + +/** Extra factory methods and pattern matchers for Decimals */ +object DecimalType { + val Unlimited: DecimalType = DecimalType(None) + + object Fixed { + def unapply(t: DecimalType): Option[(Int, Int)] = + t.precisionInfo.map(p => (p.precision, p.scale)) + } + + object Expression { + def unapply(e: Expression): Option[(Int, Int)] = e.dataType match { + case t: DecimalType => t.precisionInfo.map(p => (p.precision, p.scale)) + case _ => None + } + } + + def apply(): DecimalType = Unlimited + + def apply(precision: Int, scale: Int): DecimalType = + DecimalType(Some(PrecisionInfo(precision, scale))) + + def unapply(t: DataType): Boolean = t.isInstanceOf[DecimalType] + + def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[DecimalType] + + def isFixed(dataType: DataType): Boolean = dataType match { + case DecimalType.Fixed(_, _) => true + case _ => false + } +} + + +/** + * :: DeveloperApi :: + * + * The data type representing `Double` values. Please use the singleton [[DataTypes.DoubleType]]. + * + * @group dataType + */ +@DeveloperApi +case object DoubleType extends FractionalType { + private[sql] type JvmType = Double + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Double]] + private[sql] val fractional = implicitly[Fractional[Double]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val asIntegral = DoubleAsIfIntegral +} + + +/** + * :: DeveloperApi :: + * + * The data type representing `Float` values. Please use the singleton [[DataTypes.FloatType]]. + * + * @group dataType + */ +@DeveloperApi +case object FloatType extends FractionalType { + private[sql] type JvmType = Float + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Float]] + private[sql] val fractional = implicitly[Fractional[Float]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val asIntegral = FloatAsIfIntegral +} + + +object ArrayType { + /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ + def apply(elementType: DataType): ArrayType = ArrayType(elementType, true) +} + + +/** + * :: DeveloperApi :: + * + * The data type for collections of multiple values. + * Internally these are represented as columns that contain a ``scala.collection.Seq``. + * + * Please use [[DataTypes.createArrayType()]] to create a specific instance. + * + * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and + * `containsNull: Boolean`. The field of `elementType` is used to specify the type of + * array elements. The field of `containsNull` is used to specify if the array has `null` values. + * + * @param elementType The data type of values. + * @param containsNull Indicates if values have `null` values + * + * @group dataType + */ +@DeveloperApi +case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append( + s"$prefix-- element: ${elementType.typeName} (containsNull = $containsNull)\n") + DataType.buildFormattedString(elementType, s"$prefix |", builder) + } + + override private[sql] def jsonValue = + ("type" -> typeName) ~ + ("elementType" -> elementType.jsonValue) ~ + ("containsNull" -> containsNull) +} + + +/** + * A field inside a StructType. + * + * @param name The name of this field. + * @param dataType The data type of this field. + * @param nullable Indicates if values of this field can be `null` values. + * @param metadata The metadata of this field. The metadata should be preserved during + * transformation if the content of the column is not modified, e.g, in selection. + */ +case class StructField( + name: String, + dataType: DataType, + nullable: Boolean = true, + metadata: Metadata = Metadata.empty) { + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append(s"$prefix-- $name: ${dataType.typeName} (nullable = $nullable)\n") + DataType.buildFormattedString(dataType, s"$prefix |", builder) + } + + // override the default toString to be compatible with legacy parquet files. + override def toString: String = s"StructField($name,$dataType,$nullable)" + + private[sql] def jsonValue: JValue = { + ("name" -> name) ~ + ("type" -> dataType.jsonValue) ~ + ("nullable" -> nullable) ~ + ("metadata" -> metadata.jsonValue) + } +} + + +object StructType { + protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = + StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) + + def apply(fields: Seq[StructField]): StructType = StructType(fields.toArray) + + def apply(fields: java.util.List[StructField]): StructType = { + StructType(fields.toArray.asInstanceOf[Array[StructField]]) + } +} + + +/** + * :: DeveloperApi :: + * + * A [[StructType]] object can be constructed by + * {{{ + * StructType(fields: Seq[StructField]) + * }}} + * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. + * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. + * If a provided name does not have a matching field, it will be ignored. For the case + * of extracting a single StructField, a `null` will be returned. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val struct = + * StructType( + * StructField("a", IntegerType, true) :: + * StructField("b", LongType, false) :: + * StructField("c", BooleanType, false) :: Nil) + * + * // Extract a single StructField. + * val singleField = struct("b") + * // singleField: StructField = StructField(b,LongType,false) + * + * // This struct does not have a field called "d". null will be returned. + * val nonExisting = struct("d") + * // nonExisting: StructField = null + * + * // Extract multiple StructFields. Field names are provided in a set. + * // A StructType object will be returned. + * val twoFields = struct(Set("b", "c")) + * // twoFields: StructType = + * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * + * // Any names without matching fields will be ignored. + * // For the case shown below, "d" will be ignored and + * // it is treated as struct(Set("b", "c")). + * val ignoreNonExisting = struct(Set("b", "c", "d")) + * // ignoreNonExisting: StructType = + * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * }}} + * + * A [[org.apache.spark.sql.catalyst.expressions.Row]] object is used as a value of the StructType. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val innerStruct = + * StructType( + * StructField("f1", IntegerType, true) :: + * StructField("f2", LongType, false) :: + * StructField("f3", BooleanType, false) :: Nil) + * + * val struct = StructType( + * StructField("a", innerStruct, true) :: Nil) + * + * // Create a Row with the schema defined by struct + * val row = Row(Row(1, 2, true)) + * // row: Row = [[1,2,true]] + * }}} + * + * @group dataType + */ +@DeveloperApi +case class StructType(fields: Array[StructField]) extends DataType with Seq[StructField] { + + /** Returns all field names in an array. */ + def fieldNames: Array[String] = fields.map(_.name) + + private lazy val fieldNamesSet: Set[String] = fieldNames.toSet + private lazy val nameToField: Map[String, StructField] = fields.map(f => f.name -> f).toMap + + /** + * Extracts a [[StructField]] of the given name. If the [[StructType]] object does not + * have a name matching the given name, `null` will be returned. + */ + def apply(name: String): StructField = { + nameToField.getOrElse(name, throw new IllegalArgumentException(s"Field $name does not exist.")) + } + + /** + * Returns a [[StructType]] containing [[StructField]]s of the given names, preserving the + * original order of fields. Those names which do not have matching fields will be ignored. + */ + def apply(names: Set[String]): StructType = { + val nonExistFields = names -- fieldNamesSet + if (nonExistFields.nonEmpty) { + throw new IllegalArgumentException( + s"Field ${nonExistFields.mkString(",")} does not exist.") + } + // Preserve the original order of fields. + StructType(fields.filter(f => names.contains(f.name))) + } + + protected[sql] def toAttributes: Seq[AttributeReference] = + map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) + + def treeString: String = { + val builder = new StringBuilder + builder.append("root\n") + val prefix = " |" + fields.foreach(field => field.buildFormattedString(prefix, builder)) + + builder.toString() + } + + def printTreeString(): Unit = println(treeString) + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + fields.foreach(field => field.buildFormattedString(prefix, builder)) + } + + override private[sql] def jsonValue = + ("type" -> typeName) ~ + ("fields" -> map(_.jsonValue)) + + override def apply(fieldIndex: Int): StructField = fields(fieldIndex) + + override def length: Int = fields.length + + override def iterator: Iterator[StructField] = fields.iterator +} + + +object MapType { + /** + * Construct a [[MapType]] object with the given key type and value type. + * The `valueContainsNull` is true. + */ + def apply(keyType: DataType, valueType: DataType): MapType = + MapType(keyType: DataType, valueType: DataType, valueContainsNull = true) +} + + +/** + * :: DeveloperApi :: + * + * The data type for Maps. Keys in a map are not allowed to have `null` values. + * + * Please use [[DataTypes.createMapType()]] to create a specific instance. + * + * @param keyType The data type of map keys. + * @param valueType The data type of map values. + * @param valueContainsNull Indicates if map values have `null` values. + * + * @group dataType + */ +case class MapType( + keyType: DataType, + valueType: DataType, + valueContainsNull: Boolean) extends DataType { + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append(s"$prefix-- key: ${keyType.typeName}\n") + builder.append(s"$prefix-- value: ${valueType.typeName} " + + s"(valueContainsNull = $valueContainsNull)\n") + DataType.buildFormattedString(keyType, s"$prefix |", builder) + DataType.buildFormattedString(valueType, s"$prefix |", builder) + } + + override private[sql] def jsonValue: JValue = + ("type" -> typeName) ~ + ("keyType" -> keyType.jsonValue) ~ + ("valueType" -> valueType.jsonValue) ~ + ("valueContainsNull" -> valueContainsNull) +} + + +/** + * ::DeveloperApi:: + * The data type for User Defined Types (UDTs). + * + * This interface allows a user to make their own classes more interoperable with SparkSQL; + * e.g., by creating a [[UserDefinedType]] for a class X, it becomes possible to create + * a SchemaRDD which has class X in the schema. + * + * For SparkSQL to recognize UDTs, the UDT must be annotated with + * [[SQLUserDefinedType]]. + * + * The conversion via `serialize` occurs when instantiating a `SchemaRDD` from another RDD. + * The conversion via `deserialize` occurs when reading from a `SchemaRDD`. + */ +@DeveloperApi +abstract class UserDefinedType[UserType] extends DataType with Serializable { + + /** Underlying storage type for this UDT */ + def sqlType: DataType + + /** Paired Python UDT class, if exists. */ + def pyUDT: String = null + + /** + * Convert the user type to a SQL datum + * + * TODO: Can we make this take obj: UserType? The issue is in ScalaReflection.convertToCatalyst, + * where we need to convert Any to UserType. + */ + def serialize(obj: Any): Any + + /** Convert a SQL datum to the user type */ + def deserialize(datum: Any): UserType + + override private[sql] def jsonValue: JValue = { + ("type" -> "udt") ~ + ("class" -> this.getClass.getName) ~ + ("pyClass" -> pyUDT) ~ + ("sqlType" -> sqlType.jsonValue) + } + + /** + * Class object for the UserType + */ + def userClass: java.lang.Class[UserType] +} 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 similarity index 99% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/decimal/Decimal.scala index 708362acf32dc..c7864d1ae9e74 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/core/src/main/java/org/apache/spark/sql/api/java/NullType.java b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/package.scala similarity index 76% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/package.scala index 6d5ecdf46e551..346a51ea10c82 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/package.scala @@ -15,13 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java; +package org.apache.spark.sql /** - * The data type representing null and NULL values. - * - * {@code NullType} is represented by the singleton object {@link DataType#NullType}. + * Contains a type system for attributes produced by relations, including complex types like + * structs, arrays and maps. */ -public class NullType extends DataType { - protected NullType() {} -} +package object types 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 7be24bea7d5a6..117725df32e19 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 82f2101d8ce17..0f8d165d1c49a 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 3677a6e72e23a..587785ccfa0f1 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 dfa2d958c0faf..f5a502b43f80b 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 4ba7d87ba8c5a..8552448b8d108 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 0a27cce337482..9fdf3efa02bb6 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 017b180c574b4..da912ab382179 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 036fd3fa1d6a1..cdb843f959704 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 f005b7df21043..d7d60efee50fa 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 similarity index 98% rename from sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index e9740d913cf57..892195f46ea24 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 similarity index 99% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala index e32f1ac382130..813377df00132 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 b73a371e93001..0000000000000 --- 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 7daad60f62a0b..0000000000000 --- 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 5a1f52725631b..0000000000000 --- 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 e5cdf06b21bbe..0000000000000 --- 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 6677793baa365..0000000000000 --- 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 60752451ecfc7..0000000000000 --- 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 f0060d0bcf9f5..0000000000000 --- 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 4a6a37f69176a..0000000000000 --- 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 bfd70490bbbbb..0000000000000 --- 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 af13a46eb165c..0000000000000 --- 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 063e6b34abc48..0000000000000 --- 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 0f819fb01a76a..0000000000000 --- 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 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 6e6b12f0722c5..0000000000000 --- 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/ShortType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java deleted file mode 100644 index 7d7604b4e3d2d..0000000000000 --- 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 f4ba0c07c9c6e..0000000000000 --- 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 7c60d492bcdf0..0000000000000 --- 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 a4b501efd9a10..0000000000000 --- 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 06d44c731cdfe..0000000000000 --- 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 f0d079d25b5d4..0000000000000 --- 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 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 that = (UserDefinedType) 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 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 6a1a4d995bf61..917b079d70791 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 @@ -21,7 +21,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 @@ -29,14 +28,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 80787b61ce1bf..686bcdfbb4ff1 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/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 8884204e5079f..deb2589758d4a 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.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 5b9c612487ace..9e10e532fb011 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 401798e317e96..207e2805fffe3 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 158f26e3d445f..4186c274515ab 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 a7d0f4f127ecc..0000000000000 --- 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 538dd5b734664..91c4c105b14e6 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 c68dceef3b142..3a4977b836af7 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 668efe4a3b2a8..391b3dae5c8ce 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 ab66c85c4f242..fcf2faa0914c0 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 27ac5f4dbdbbc..7dff9deac8dc0 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 628d9cec41d6b..aead768ecdf0a 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 acb06cb5376b4..879d29bcfa6f6 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 29edcf17242c5..64673248394c6 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 d2d8cb1c62d40..09668b3ccc4db 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 7c3bf947e743d..4abe26fe4afc6 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 84d96e612f0dc..131146012ecac 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 ce878c137e627..677a8d2a3db51 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,15 +17,15 @@ package org.apache.spark.sql.execution -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.{SQLContext, Strategy, execution} 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 61be5ed2db65c..46245cd5a1869 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 5a41399971dda..741ccb8fb8fa7 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/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 00449c200704a..c92ec543e2936 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 1fd8e6220f83b..b75266d5aa409 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 @@ -116,358 +116,9 @@ package object sql { @DeveloperApi 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 1bbb66aaa19a3..7f437c40777f2 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 56e7d11b2feea..f08350878f239 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 9049eb5932b79..af7248fdf451d 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 97447871a11ee..6d8c682ccced8 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: *
    @@ -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: *
      *
    • Primitive types are converted into Parquet's primitive types.
    • - *
    • [[org.apache.spark.sql.catalyst.types.StructType]]s are converted + *
    • [[org.apache.spark.sql.types.StructType]]s are converted * into Parquet's `GroupType` with the corresponding field types.
    • - *
    • [[org.apache.spark.sql.catalyst.types.ArrayType]]s are converted + *
    • [[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.
    • - *
    • [[org.apache.spark.sql.catalyst.types.MapType]]s are converted + *
    • [[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 2e0c6c51c00e5..109bed7223075 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 4d87f6817dcb9..c11759704c2d4 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,17 @@ private[sql] case class LogicalRelation(relation: BaseRelation) extends LeafNode with MultiInstanceRelation { - override val output = relation.schema.toAttributes + override val output: Seq[AttributeReference] = { + // If the external data source was written against Spark 1.2.0, the schema method returns + // the old data type system. Convert it into the new type system and then return the attributes. + val schemaMethod = relation.getClass.getDeclaredMethod("schema") + schemaMethod.invoke(relation) match { + case oldSchemaType: org.apache.spark.sql.catalyst.types.StructType => + oldSchemaType.toNewType.toAttributes + case newSchemaType: org.apache.spark.sql.types.StructType => + newSchemaType.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/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 02eff80456dbe..78ef2c1c6ecb8 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.{SQLConf, 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 b9569e96c0312..006b16fbe07bd 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 d4ef517981699..0000000000000 --- 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 a9a11285def54..88017eb47d908 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 @@ 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 void udf1Test() { 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 void udf2Test() { 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 a04b8060cd658..de586ba635913 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.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 Row call(Person person) throws Exception { }); List fields = new ArrayList(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 void applySchemaToJSON() { "\"bigInteger\":92233720368547758069, \"double\":1.7976931348623157E305, " + "\"boolean\":false, \"null\":null}")); List fields = new ArrayList(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 expectedResult = new ArrayList(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 8396a29c61c4c..0000000000000 --- 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 simpleFields = new ArrayList(); - 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 complexFields = new ArrayList(); - 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 simpleFields = new ArrayList(); - 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 simpleFields = new ArrayList(); - 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 ab88f3ad10d66..efe622f8bcb27 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 811319e0a6601..f5b945f468dad 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 add4e218a22ee..d4c27f52f7abd 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._ @@ -731,7 +732,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 cf3a59e545905..40fb8d5779fdf 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 1806a1dd82023..a0d54d17f5f13 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 8afc3a9fb2187..fdbb4282baf48 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 62fe59dd345d7..0000000000000 --- 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 a9f0851f8826c..9be0b38e689ff 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 3f3f35d50188b..87e608a8853dc 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 a1f21219eaf2f..f941465fa3e35 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 21906e3fdcc6f..f95c895587f3f 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 cb73f3da81e24..80bd5c94570cb 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 1cdb909146d57..c82d9799359c7 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 73f31c0233343..88011631ee4e3 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 4ce2552112c92..08df1db375097 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 7db723d648d80..0b18b4119268f 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 8dce3372a8db3..4086bb6d498f8 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 10a01474e95be..be70d156732cd 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 a5fe2e8da2840..d1df6e2f9db05 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 34d61bf908483..64274950b868e 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 939b3c0c66de7..390538d35a348 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 fee2e22611cdc..7900b3e8948d9 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 3cd7b0115d567..55b58c96a5343 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import org.apache.spark.sql._ +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 5550183621fb6..908355d31a99d 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 @@ -34,9 +34,9 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.Logging import org.apache.spark.sql.catalyst.plans.logical.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.types._ import org.apache.spark.sql.{SQLConf, SchemaRDD, Row => SparkRow} /** 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 798a690a20427..850ce49178ad8 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 @@ -32,7 +32,7 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.Logging import org.apache.spark.sql.catalyst.plans.logical.SetCommand -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.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} 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 982e0593fcfd1..32e9072b825cb 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,11 +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.catalyst.types.decimal.Decimal -import org.apache.spark.sql.execution.{SparkPlan, ExecutedCommand, ExtractPythonUdfs, QueryExecutionException} +import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUdfs, 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 a156d6f7e285d..245b847cf4cd4 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 b31a3ec25096b..6cd1b54867ae2 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 @@ -43,7 +43,7 @@ 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.types._ import org.apache.spark.util.Utils /* Implicit conversions */ @@ -431,7 +431,7 @@ object HiveMetastoreTypes extends RegexParsers { protected lazy val structType: Parser[DataType] = "struct" ~> "<" ~> repsep(structField,",") <~ ">" ^^ { - case fields => new StructType(fields) + case fields => StructType(fields) } protected lazy val dataType: Parser[DataType] = 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 8a9613cf96e54..3e2637af0abf1 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,9 +32,9 @@ 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.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 d3f6381b69a4d..61f1f8cac937d 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 @@ -24,11 +24,11 @@ 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._ 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.types.StringType import org.apache.spark.sql.{SQLContext, SchemaRDD, Strategy} import scala.collection.JavaConversions._ 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 8ba818af5f9d0..781a2e9164c82 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 8bbcd6fec1f3b..b56175fe76376 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/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 7d863f9d89dae..d898b876c39f8 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 f90d3607915ae..dc23d9a101d17 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 86535f8dd4f58..aad48ada52642 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,8 +19,8 @@ package org.apache.spark.sql.hive import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.types.StructType 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 fb481edc853b7..7cfb875e05db3 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/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 2d01a85067518..1a3d3e42167d2 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 class HiveFunctionWrapper(var 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 b78c75798e988..33ff0ada91eab 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 /**