Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,14 @@

package org.apache.spark.sql.catalyst

import java.sql.{Date, Timestamp}
import java.sql.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.date.Date
import org.apache.spark.sql.catalyst.types.decimal.Decimal


Expand Down Expand Up @@ -140,6 +141,7 @@ trait ScalaReflection {
valueDataType, valueContainsNull = valueNullable), 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.Date] => Schema(DateType, nullable = true)
case t if t <:< typeOf[Date] => Schema(DateType, nullable = true)
case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true)
case t if t <:< typeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,7 @@

package org.apache.spark.sql.catalyst

import java.sql.{Date, Timestamp}

import org.apache.spark.sql.catalyst.types.decimal.Decimal
import java.sql.Timestamp

import scala.language.implicitConversions
import scala.reflect.runtime.universe.{TypeTag, typeTag}
Expand All @@ -29,6 +27,8 @@ 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.date.Date
import org.apache.spark.sql.catalyst.types.decimal.Decimal

/**
* A collection of implicit conversions that create a DSL for constructing catalyst data structures.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,12 @@

package org.apache.spark.sql.catalyst.expressions

import java.sql.{Date, Timestamp}
import java.sql.Timestamp
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.date.Date
import org.apache.spark.sql.catalyst.types.decimal.Decimal

/** Cast the child expression to the target data type. */
Expand Down Expand Up @@ -113,7 +113,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
case ByteType =>
buildCast[Byte](_, b => new Timestamp(b))
case DateType =>
buildCast[Date](_, d => new Timestamp(d.getTime))
buildCast[Date](_, d => new Timestamp(d.toLong))
// TimestampWritable.decimalToTimestamp
case DecimalType() =>
buildCast[Decimal](_, d => decimalToTimestamp(d))
Expand Down Expand Up @@ -166,21 +166,16 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
}
}

// Converts Timestamp to string according to Hive TimestampWritable convention
private[this] def timestampToDateString(ts: Timestamp): String = {
Cast.threadLocalDateFormat.get.format(ts)
}

// DateConverter
private[this] def castToDate: Any => Any = child.dataType match {
case StringType =>
buildCast[String](_, s =>
try Date.valueOf(s) catch { case _: java.lang.IllegalArgumentException => null }
try Date(s) catch { case _: java.lang.IllegalArgumentException => null }
)
case TimestampType =>
// throw valid precision more than seconds, according to Hive.
// Timestamp.nanos is in 0 to 999,999,999, no more than a second.
buildCast[Timestamp](_, t => new Date(Math.floor(t.getTime / 1000.0).toLong * 1000))
buildCast[Timestamp](_, t => Date(Math.floor(t.getTime / 1000.0).toLong * 1000))
// Hive throws this exception as a Semantic Exception
// It is never possible to compare result when hive return with exception, so we can return null
// NULL is more reasonable here, since the query itself obeys the grammar.
Expand All @@ -195,7 +190,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w

// Converts Date to string according to Hive DateWritable convention
private[this] def dateToString(d: Date): String = {
Cast.threadLocalDateFormat.get.format(d)
d.toString
}

// LongConverter
Expand Down Expand Up @@ -368,13 +363,6 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
}

object Cast {
// `SimpleDateFormat` is not thread-safe.
private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] {
override def initialValue() = {
new SimpleDateFormat("yyyy-MM-dd")
}
}

// `SimpleDateFormat` is not thread-safe.
private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] {
override def initialValue() = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,11 @@

package org.apache.spark.sql.catalyst.expressions

import java.sql.{Date, Timestamp}
import java.sql.Timestamp

import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.catalyst.types.decimal.Decimal
import org.apache.spark.sql.catalyst.types.date.Date

object Literal {
def apply(v: Any): Literal = v match {
Expand All @@ -35,6 +36,7 @@ object Literal {
case d: BigDecimal => Literal(Decimal(d), DecimalType.Unlimited)
case d: Decimal => Literal(d, DecimalType.Unlimited)
case t: Timestamp => Literal(t, TimestampType)
case d: java.sql.Date => Literal(Date(d), DateType)
case d: Date => Literal(d, DateType)
case a: Array[Byte] => Literal(a, BinaryType)
case null => Literal(null, NullType)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,9 @@

package org.apache.spark.sql.catalyst.types

import java.sql.{Date, Timestamp}
import java.sql.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
Expand All @@ -33,6 +33,7 @@ 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.types.date._
import org.apache.spark.sql.catalyst.util.Metadata
import org.apache.spark.util.Utils

Expand Down Expand Up @@ -302,9 +303,7 @@ case object DateType extends NativeType {

@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)
}
private[sql] val ordering = implicitly[Ordering[JvmType]]
}

abstract class NumericType extends NativeType with PrimitiveType {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.catalyst.types.date

import java.util.{Calendar, TimeZone}

/**
* A mutable implementation of java.sql.Date that holds an Int for days since epoch, like Hive.
*/
final class Date extends Ordered[Date] with Serializable {
private var daysSinceEpoch: Int = 0

/**
* Set this Decimal to the given Long. Will have precision 20 and scale 0.
*/
def set(days: Int): Date = {
this.daysSinceEpoch = days
this
}

/**
* Set this Decimal to the given Int. Will have precision 10 and scale 0.
*/
def set(days: Long): Date = {
this.daysSinceEpoch = days.toInt
this
}

/**
* Set this Decimal to the given BigDecimal value, inheriting its precision and scale.
*/
def set(date: Date): Date = {
this.daysSinceEpoch = date.daysSinceEpoch
this
}

def toDays: Int = {
this.daysSinceEpoch
}

def toJavaDate: java.sql.Date = {
new java.sql.Date(toLong)
}

override def toString: String = toJavaDate.toString()

def toLong: Long = {
val millisUtc = daysSinceEpoch.toLong * Date.MILLIS_PER_DAY
millisUtc - Date.LOCAL_TIMEZONE.get().getOffset(millisUtc)
}

def toInt: Int = toLong.toInt

def toShort: Short = toLong.toShort

def toByte: Byte = toLong.toByte

override def clone(): Date = new Date().set(this)

override def compare(other: Date): Int = {
daysSinceEpoch.compareTo(other.daysSinceEpoch)
}

override def equals(other: Any) = other match {
case d: Date =>
compare(d) == 0
case _ =>
false
}

override def hashCode(): Int = daysSinceEpoch
}

object Date {
private val MILLIS_PER_DAY = 86400000

// Java TimeZone has no mention of thread safety. Use thread local instance to be safe.
private val LOCAL_TIMEZONE = new ThreadLocal[TimeZone] {
override protected def initialValue: TimeZone = {
Calendar.getInstance.getTimeZone
}
}

private def javaDateToDays(d: java.sql.Date): Int = {
millisToDays(d.getTime)
}

private def millisToDays(millisLocal: Long): Int = {
((millisLocal + LOCAL_TIMEZONE.get().getOffset(millisLocal)) / MILLIS_PER_DAY).toInt
}

def apply(value: java.sql.Date): Date = new Date().set(javaDateToDays(value))

def apply(value: Int): Date = new Date().set(millisToDays(value))

def apply(value: Long): Date = new Date().set(millisToDays(value))

def apply(value: String): Date = new Date().set(javaDateToDays(java.sql.Date.valueOf(value)))

}
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,9 @@
package org.apache.spark.sql.catalyst

import java.math.BigInteger
import java.sql.{Date, Timestamp}
import java.sql.Timestamp

import org.apache.spark.sql.catalyst.types.date.Date
import org.scalatest.FunSuite

import org.apache.spark.sql.catalyst.expressions.Row
Expand Down Expand Up @@ -203,7 +204,7 @@ class ScalaReflectionSuite extends FunSuite {
assert(DecimalType.Unlimited === typeOfObject(BigDecimal("1.7976931348623157E318")))

// DateType
assert(DateType === typeOfObject(Date.valueOf("2014-07-25")))
assert(DateType === typeOfObject(Date("2014-07-25")))

// TimestampType
assert(TimestampType === typeOfObject(Timestamp.valueOf("2014-07-25 10:26:00")))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,12 @@

package org.apache.spark.sql.catalyst.expressions

import java.sql.{Date, Timestamp}
import java.sql.Timestamp

import scala.collection.immutable.HashSet

import org.apache.spark.sql.catalyst.types.decimal.Decimal
import org.apache.spark.sql.catalyst.types.date.Date
import org.scalatest.FunSuite
import org.scalatest.Matchers._
import org.scalactic.TripleEqualsSupport.Spread
Expand Down Expand Up @@ -260,7 +261,8 @@ class ExpressionEvaluationSuite extends FunSuite {
test("data type casting") {

val sd = "1970-01-01"
val d = Date.valueOf(sd)
val d = Date(sd)
val zts = sd + " 00:00:00"
val sts = sd + " 00:00:02"
val nts = sts + ".1"
val ts = Timestamp.valueOf(nts)
Expand All @@ -284,7 +286,7 @@ class ExpressionEvaluationSuite extends FunSuite {
checkEvaluation(
Cast(Cast(Literal(nts) cast TimestampType, DateType), StringType), sd)
checkEvaluation(
Cast(Cast(Literal(ts) cast DateType, TimestampType), StringType), sts)
Cast(Cast(Literal(ts) cast DateType, TimestampType), StringType), zts)

checkEvaluation(Cast("abdef" cast BinaryType, StringType), "abdef")

Expand Down Expand Up @@ -335,8 +337,8 @@ class ExpressionEvaluationSuite extends FunSuite {
}

test("date") {
val d1 = Date.valueOf("1970-01-01")
val d2 = Date.valueOf("1970-01-02")
val d1 = Date("1970-01-01")
val d2 = Date("1970-01-02")
checkEvaluation(Literal(d1) < Literal(d2), true)
}

Expand Down Expand Up @@ -416,16 +418,16 @@ class ExpressionEvaluationSuite extends FunSuite {
}

test("date casting") {
val d = Date.valueOf("1970-01-01")
checkEvaluation(Cast(d, ShortType), null)
checkEvaluation(Cast(d, IntegerType), null)
checkEvaluation(Cast(d, LongType), null)
checkEvaluation(Cast(d, FloatType), null)
checkEvaluation(Cast(d, DoubleType), null)
checkEvaluation(Cast(d, DecimalType.Unlimited), null)
checkEvaluation(Cast(d, DecimalType(10, 2)), null)
checkEvaluation(Cast(d, StringType), "1970-01-01")
checkEvaluation(Cast(Cast(d, TimestampType), StringType), "1970-01-01 00:00:00")
val d = Date("1970-01-01")
checkEvaluation(Cast(Literal(d), ShortType), null)
checkEvaluation(Cast(Literal(d), IntegerType), null)
checkEvaluation(Cast(Literal(d), LongType), null)
checkEvaluation(Cast(Literal(d), FloatType), null)
checkEvaluation(Cast(Literal(d), DoubleType), null)
checkEvaluation(Cast(Literal(d), DecimalType.Unlimited), null)
checkEvaluation(Cast(Literal(d), DecimalType(10, 2)), null)
checkEvaluation(Cast(Literal(d), StringType), "1970-01-01")
checkEvaluation(Cast(Cast(Literal(d), TimestampType), StringType), "1970-01-01 00:00:00")
}

test("timestamp casting") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.api.java;

/**
* The data type representing java.sql.Date values.
* The data type representing int values of daysSinceEpoch.
*
* {@code DateType} is represented by the singleton object {@link DataType#DateType}.
*/
Expand Down
Loading