aboutsummaryrefslogtreecommitdiff
path: root/sql/catalyst/src
diff options
context:
space:
mode:
authorReynold Xin <rxin@databricks.com>2015-04-23 01:43:40 -0700
committerReynold Xin <rxin@databricks.com>2015-04-23 01:43:40 -0700
commitf60bece14f98450b4a71b00d7b58525f06e1f9ed (patch)
tree7c60f98f401d49bb3069035f6f585f8648b982f7 /sql/catalyst/src
parent29163c520087e89ca322521db1dd8656d86a6f0e (diff)
downloadspark-f60bece14f98450b4a71b00d7b58525f06e1f9ed.tar.gz
spark-f60bece14f98450b4a71b00d7b58525f06e1f9ed.tar.bz2
spark-f60bece14f98450b4a71b00d7b58525f06e1f9ed.zip
[SPARK-7069][SQL] Rename NativeType -> AtomicType.
Also renamed JvmType to InternalType. Author: Reynold Xin <rxin@databricks.com> Closes #5651 from rxin/native-to-atomic-type and squashes the following commits: cbd4028 [Reynold Xin] [SPARK-7069][SQL] Rename NativeType -> AtomicType.
Diffstat (limited to 'sql/catalyst/src')
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala24
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala18
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala10
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala6
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala114
7 files changed, 93 insertions, 87 deletions
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 d9521953ca..c52965507c 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
@@ -17,8 +17,6 @@
package org.apache.spark.sql.catalyst
-import java.sql.Timestamp
-
import org.apache.spark.util.Utils
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
@@ -110,7 +108,7 @@ trait ScalaReflection {
StructField(p.name.toString, dataType, nullable)
}), nullable = true)
case t if t <:< typeOf[String] => Schema(StringType, nullable = true)
- case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true)
+ case t if t <:< typeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true)
case t if t <:< typeOf[java.sql.Date] => Schema(DateType, nullable = true)
case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true)
case t if t <:< typeOf[java.math.BigDecimal] => Schema(DecimalType.Unlimited, nullable = true)
@@ -136,20 +134,20 @@ trait ScalaReflection {
def typeOfObject: PartialFunction[Any, DataType] = {
// The data type can be determined without ambiguity.
- case obj: BooleanType.JvmType => BooleanType
- case obj: BinaryType.JvmType => BinaryType
+ case obj: Boolean => BooleanType
+ case obj: Array[Byte] => BinaryType
case obj: String => StringType
- case obj: StringType.JvmType => StringType
- case obj: ByteType.JvmType => ByteType
- case obj: ShortType.JvmType => ShortType
- case obj: IntegerType.JvmType => IntegerType
- case obj: LongType.JvmType => LongType
- case obj: FloatType.JvmType => FloatType
- case obj: DoubleType.JvmType => DoubleType
+ case obj: UTF8String => StringType
+ case obj: Byte => ByteType
+ case obj: Short => ShortType
+ case obj: Int => IntegerType
+ case obj: Long => LongType
+ case obj: Float => FloatType
+ case obj: Double => DoubleType
case obj: java.sql.Date => DateType
case obj: java.math.BigDecimal => DecimalType.Unlimited
case obj: Decimal => DecimalType.Unlimited
- case obj: TimestampType.JvmType => TimestampType
+ case obj: java.sql.Timestamp => TimestampType
case null => NullType
// For other cases, there is no obvious mapping from the type of the given object to a
// Catalyst data type. A user should provide his/her specific rules
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 566b34f7c3..140ccd8d37 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
@@ -346,7 +346,7 @@ case class MaxOf(left: Expression, right: Expression) extends Expression {
}
lazy val ordering = left.dataType match {
- case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]]
+ case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]]
case other => sys.error(s"Type $other does not support ordered operations")
}
@@ -391,7 +391,7 @@ case class MinOf(left: Expression, right: Expression) extends Expression {
}
lazy val ordering = left.dataType match {
- case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]]
+ case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]]
case other => sys.error(s"Type $other does not support ordered operations")
}
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 eeffedb558..cbe5203473 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
@@ -623,7 +623,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin
protected def getColumn(inputRow: TermName, dataType: DataType, ordinal: Int) = {
dataType match {
case StringType => q"$inputRow($ordinal).asInstanceOf[org.apache.spark.sql.types.UTF8String]"
- case dt @ NativeType() => q"$inputRow.${accessorForType(dt)}($ordinal)"
+ case dt: DataType if isNativeType(dt) => q"$inputRow.${accessorForType(dt)}($ordinal)"
case _ => q"$inputRow.apply($ordinal).asInstanceOf[${termForType(dataType)}]"
}
}
@@ -635,7 +635,8 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin
value: TermName) = {
dataType match {
case StringType => q"$destinationRow.update($ordinal, $value)"
- case dt @ NativeType() => q"$destinationRow.${mutatorForType(dt)}($ordinal, $value)"
+ case dt: DataType if isNativeType(dt) =>
+ q"$destinationRow.${mutatorForType(dt)}($ordinal, $value)"
case _ => q"$destinationRow.update($ordinal, $value)"
}
}
@@ -675,7 +676,18 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin
}
protected def termForType(dt: DataType) = dt match {
- case n: NativeType => n.tag
+ case n: AtomicType => n.tag
case _ => typeTag[Any]
}
+
+ /**
+ * List of data types that have special accessors and setters in [[Row]].
+ */
+ protected val nativeTypes =
+ Seq(IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType)
+
+ /**
+ * Returns true if the data type has a special accessor and setter in [[Row]].
+ */
+ protected def isNativeType(dt: DataType) = nativeTypes.contains(dt)
}
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 d491babc2b..584f938445 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
@@ -109,7 +109,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
q"override def update(i: Int, value: Any): Unit = { ..$cases; $accessorFailure }"
}
- val specificAccessorFunctions = NativeType.all.map { dataType =>
+ val specificAccessorFunctions = nativeTypes.map { dataType =>
val ifStatements = expressions.zipWithIndex.flatMap {
// getString() is not used by expressions
case (e, i) if e.dataType == dataType && dataType != StringType =>
@@ -135,7 +135,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
}
}
- val specificMutatorFunctions = NativeType.all.map { dataType =>
+ val specificMutatorFunctions = nativeTypes.map { dataType =>
val ifStatements = expressions.zipWithIndex.flatMap {
// setString() is not used by expressions
case (e, i) if e.dataType == dataType && dataType != StringType =>
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 46522eb9c1..9cb00cb273 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
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.types.{DataType, BinaryType, BooleanType, NativeType}
+import org.apache.spark.sql.types.{DataType, BinaryType, BooleanType, AtomicType}
object InterpretedPredicate {
def create(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) =
@@ -211,7 +211,7 @@ case class LessThan(left: Expression, right: Expression) extends BinaryCompariso
s"Types do not match ${left.dataType} != ${right.dataType}")
}
left.dataType match {
- case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]]
+ case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]]
case other => sys.error(s"Type $other does not support ordered operations")
}
}
@@ -240,7 +240,7 @@ case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryCo
s"Types do not match ${left.dataType} != ${right.dataType}")
}
left.dataType match {
- case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]]
+ case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]]
case other => sys.error(s"Type $other does not support ordered operations")
}
}
@@ -269,7 +269,7 @@ case class GreaterThan(left: Expression, right: Expression) extends BinaryCompar
s"Types do not match ${left.dataType} != ${right.dataType}")
}
left.dataType match {
- case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]]
+ case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]]
case other => sys.error(s"Type $other does not support ordered operations")
}
}
@@ -298,7 +298,7 @@ case class GreaterThanOrEqual(left: Expression, right: Expression) extends Binar
s"Types do not match ${left.dataType} != ${right.dataType}")
}
left.dataType match {
- case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]]
+ case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]]
case other => sys.error(s"Type $other does not support ordered operations")
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
index 981373477a..5fd892c42e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.sql.types.{UTF8String, DataType, StructType, NativeType}
+import org.apache.spark.sql.types.{UTF8String, DataType, StructType, AtomicType}
/**
* An extended interface to [[Row]] that allows the values for each column to be updated. Setting
@@ -227,9 +227,9 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] {
return if (order.direction == Ascending) 1 else -1
} else {
val comparison = order.dataType match {
- case n: NativeType if order.direction == Ascending =>
+ case n: AtomicType if order.direction == Ascending =>
n.ordering.asInstanceOf[Ordering[Any]].compare(left, right)
- case n: NativeType if order.direction == Descending =>
+ case n: AtomicType if order.direction == Descending =>
n.ordering.asInstanceOf[Ordering[Any]].reverse.compare(left, right)
case other => sys.error(s"Type $other does not support ordered operations")
}
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
index 42e26e0599..87c7b75993 100644
--- 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
@@ -42,7 +42,8 @@ object DataType {
def fromJson(json: String): DataType = parseDataType(parse(json))
private val nonDecimalNameToType = {
- (Seq(NullType, DateType, TimestampType, BinaryType) ++ NativeType.all)
+ Seq(NullType, DateType, TimestampType, BinaryType,
+ IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType)
.map(t => t.typeName -> t).toMap
}
@@ -309,22 +310,17 @@ class NullType private() extends DataType {
case object NullType extends NullType
-protected[sql] object NativeType {
- val all = Seq(
- IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType)
-
- def unapply(dt: DataType): Boolean = all.contains(dt)
-}
-
-
-protected[sql] abstract class NativeType extends DataType {
- private[sql] type JvmType
- @transient private[sql] val tag: TypeTag[JvmType]
- private[sql] val ordering: Ordering[JvmType]
+/**
+ * An internal type used to represent everything that is not null, UDTs, arrays, structs, and maps.
+ */
+protected[sql] abstract class AtomicType extends DataType {
+ private[sql] type InternalType
+ @transient private[sql] val tag: TypeTag[InternalType]
+ private[sql] val ordering: Ordering[InternalType]
@transient private[sql] val classTag = ScalaReflectionLock.synchronized {
val mirror = runtimeMirror(Utils.getSparkClassLoader)
- ClassTag[JvmType](mirror.runtimeClass(tag.tpe))
+ ClassTag[InternalType](mirror.runtimeClass(tag.tpe))
}
}
@@ -336,13 +332,13 @@ protected[sql] abstract class NativeType extends DataType {
* @group dataType
*/
@DeveloperApi
-class StringType private() extends NativeType {
+class StringType private() extends AtomicType {
// The companion object and this class is separated so the companion object also subclasses
// this type. Otherwise, the companion object would be of type "StringType$" in byte code.
// Defined with a private constructor so the companion object is the only possible instantiation.
- private[sql] type JvmType = UTF8String
- @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
- private[sql] val ordering = implicitly[Ordering[JvmType]]
+ private[sql] type InternalType = UTF8String
+ @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
+ private[sql] val ordering = implicitly[Ordering[InternalType]]
/**
* The default size of a value of the StringType is 4096 bytes.
@@ -363,13 +359,13 @@ case object StringType extends StringType
* @group dataType
*/
@DeveloperApi
-class BinaryType private() extends NativeType {
+class BinaryType private() extends AtomicType {
// The companion object and this class is separated so the companion object also subclasses
// this type. Otherwise, the companion object would be of type "BinaryType$" in byte code.
// Defined with a private constructor so the companion object is the only possible instantiation.
- private[sql] type JvmType = Array[Byte]
- @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
- private[sql] val ordering = new Ordering[JvmType] {
+ private[sql] type InternalType = Array[Byte]
+ @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
+ private[sql] val ordering = new Ordering[InternalType] {
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))
@@ -397,13 +393,13 @@ case object BinaryType extends BinaryType
*@group dataType
*/
@DeveloperApi
-class BooleanType private() extends NativeType {
+class BooleanType private() extends AtomicType {
// The companion object and this class is separated so the companion object also subclasses
// this type. Otherwise, the companion object would be of type "BooleanType$" in byte code.
// Defined with a private constructor so the companion object is the only possible instantiation.
- private[sql] type JvmType = Boolean
- @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
- private[sql] val ordering = implicitly[Ordering[JvmType]]
+ private[sql] type InternalType = Boolean
+ @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
+ private[sql] val ordering = implicitly[Ordering[InternalType]]
/**
* The default size of a value of the BooleanType is 1 byte.
@@ -424,15 +420,15 @@ case object BooleanType extends BooleanType
* @group dataType
*/
@DeveloperApi
-class TimestampType private() extends NativeType {
+class TimestampType private() extends AtomicType {
// The companion object and this class is separated so the companion object also subclasses
// this type. Otherwise, the companion object would be of type "TimestampType$" in byte code.
// Defined with a private constructor so the companion object is the only possible instantiation.
- private[sql] type JvmType = Timestamp
+ private[sql] type InternalType = Timestamp
- @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+ @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
- private[sql] val ordering = new Ordering[JvmType] {
+ private[sql] val ordering = new Ordering[InternalType] {
def compare(x: Timestamp, y: Timestamp): Int = x.compareTo(y)
}
@@ -455,15 +451,15 @@ case object TimestampType extends TimestampType
* @group dataType
*/
@DeveloperApi
-class DateType private() extends NativeType {
+class DateType private() extends AtomicType {
// The companion object and this class is separated so the companion object also subclasses
// this type. Otherwise, the companion object would be of type "DateType$" in byte code.
// Defined with a private constructor so the companion object is the only possible instantiation.
- private[sql] type JvmType = Int
+ private[sql] type InternalType = Int
- @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+ @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
- private[sql] val ordering = implicitly[Ordering[JvmType]]
+ private[sql] val ordering = implicitly[Ordering[InternalType]]
/**
* The default size of a value of the DateType is 4 bytes.
@@ -482,13 +478,13 @@ case object DateType extends DateType
*
* @group dataType
*/
-abstract class NumericType extends NativeType {
+abstract class NumericType extends AtomicType {
// 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]
+ private[sql] val numeric: Numeric[InternalType]
}
@@ -507,7 +503,7 @@ protected[sql] object IntegralType {
protected[sql] sealed abstract class IntegralType extends NumericType {
- private[sql] val integral: Integral[JvmType]
+ private[sql] val integral: Integral[InternalType]
}
@@ -522,11 +518,11 @@ class LongType private() extends IntegralType {
// The companion object and this class is separated so the companion object also subclasses
// this type. Otherwise, the companion object would be of type "LongType$" in byte code.
// Defined with a private constructor so the companion object is the only possible instantiation.
- private[sql] type JvmType = Long
- @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+ private[sql] type InternalType = Long
+ @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
private[sql] val numeric = implicitly[Numeric[Long]]
private[sql] val integral = implicitly[Integral[Long]]
- private[sql] val ordering = implicitly[Ordering[JvmType]]
+ private[sql] val ordering = implicitly[Ordering[InternalType]]
/**
* The default size of a value of the LongType is 8 bytes.
@@ -552,11 +548,11 @@ class IntegerType private() extends IntegralType {
// The companion object and this class is separated so the companion object also subclasses
// this type. Otherwise, the companion object would be of type "IntegerType$" in byte code.
// Defined with a private constructor so the companion object is the only possible instantiation.
- private[sql] type JvmType = Int
- @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+ private[sql] type InternalType = Int
+ @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
private[sql] val numeric = implicitly[Numeric[Int]]
private[sql] val integral = implicitly[Integral[Int]]
- private[sql] val ordering = implicitly[Ordering[JvmType]]
+ private[sql] val ordering = implicitly[Ordering[InternalType]]
/**
* The default size of a value of the IntegerType is 4 bytes.
@@ -582,11 +578,11 @@ class ShortType private() extends IntegralType {
// The companion object and this class is separated so the companion object also subclasses
// this type. Otherwise, the companion object would be of type "ShortType$" in byte code.
// Defined with a private constructor so the companion object is the only possible instantiation.
- private[sql] type JvmType = Short
- @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+ private[sql] type InternalType = Short
+ @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
private[sql] val numeric = implicitly[Numeric[Short]]
private[sql] val integral = implicitly[Integral[Short]]
- private[sql] val ordering = implicitly[Ordering[JvmType]]
+ private[sql] val ordering = implicitly[Ordering[InternalType]]
/**
* The default size of a value of the ShortType is 2 bytes.
@@ -612,11 +608,11 @@ class ByteType private() extends IntegralType {
// The companion object and this class is separated so the companion object also subclasses
// this type. Otherwise, the companion object would be of type "ByteType$" in byte code.
// Defined with a private constructor so the companion object is the only possible instantiation.
- private[sql] type JvmType = Byte
- @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+ private[sql] type InternalType = Byte
+ @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
private[sql] val numeric = implicitly[Numeric[Byte]]
private[sql] val integral = implicitly[Integral[Byte]]
- private[sql] val ordering = implicitly[Ordering[JvmType]]
+ private[sql] val ordering = implicitly[Ordering[InternalType]]
/**
* The default size of a value of the ByteType is 1 byte.
@@ -641,8 +637,8 @@ protected[sql] object FractionalType {
protected[sql] sealed abstract class FractionalType extends NumericType {
- private[sql] val fractional: Fractional[JvmType]
- private[sql] val asIntegral: Integral[JvmType]
+ private[sql] val fractional: Fractional[InternalType]
+ private[sql] val asIntegral: Integral[InternalType]
}
@@ -665,8 +661,8 @@ case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalT
/** No-arg constructor for kryo. */
protected def this() = this(null)
- private[sql] type JvmType = Decimal
- @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+ private[sql] type InternalType = Decimal
+ @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
private[sql] val numeric = Decimal.DecimalIsFractional
private[sql] val fractional = Decimal.DecimalIsFractional
private[sql] val ordering = Decimal.DecimalIsFractional
@@ -743,11 +739,11 @@ class DoubleType private() extends FractionalType {
// The companion object and this class is separated so the companion object also subclasses
// this type. Otherwise, the companion object would be of type "DoubleType$" in byte code.
// Defined with a private constructor so the companion object is the only possible instantiation.
- private[sql] type JvmType = Double
- @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+ private[sql] type InternalType = Double
+ @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
private[sql] val numeric = implicitly[Numeric[Double]]
private[sql] val fractional = implicitly[Fractional[Double]]
- private[sql] val ordering = implicitly[Ordering[JvmType]]
+ private[sql] val ordering = implicitly[Ordering[InternalType]]
private[sql] val asIntegral = DoubleAsIfIntegral
/**
@@ -772,11 +768,11 @@ class FloatType private() extends FractionalType {
// The companion object and this class is separated so the companion object also subclasses
// this type. Otherwise, the companion object would be of type "FloatType$" in byte code.
// Defined with a private constructor so the companion object is the only possible instantiation.
- private[sql] type JvmType = Float
- @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] }
+ private[sql] type InternalType = Float
+ @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
private[sql] val numeric = implicitly[Numeric[Float]]
private[sql] val fractional = implicitly[Fractional[Float]]
- private[sql] val ordering = implicitly[Ordering[JvmType]]
+ private[sql] val ordering = implicitly[Ordering[InternalType]]
private[sql] val asIntegral = FloatAsIfIntegral
/**