Repository: spark
Updated Branches:
  refs/heads/master 4b497a724 -> 881662e9c


[SPARK-8589] [SQL] cleanup DateTimeUtils

move date time related operations into `DateTimeUtils` and rename some methods 
to make it more clear.

Author: Wenchen Fan <cloud0...@outlook.com>

Closes #6980 from cloud-fan/datetime and squashes the following commits:

9373a9d [Wenchen Fan] cleanup DateTimeUtil


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

Branch: refs/heads/master
Commit: 881662e9c93893430756320f51cef0fc6643f681
Parents: 4b497a7
Author: Wenchen Fan <cloud0...@outlook.com>
Authored: Mon Jun 29 16:34:50 2015 -0700
Committer: Michael Armbrust <mich...@databricks.com>
Committed: Mon Jun 29 16:34:50 2015 -0700

----------------------------------------------------------------------
 .../spark/sql/catalyst/expressions/Cast.scala   | 43 ++----------
 .../spark/sql/catalyst/util/DateTimeUtils.scala | 70 ++++++++++++++------
 .../spark/sql/hive/hiveWriterContainers.scala   |  2 +-
 3 files changed, 58 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/881662e9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index 8d66968..d69d490 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
@@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.expressions
 
 import java.math.{BigDecimal => JavaBigDecimal}
 import java.sql.{Date, Timestamp}
-import java.text.{DateFormat, SimpleDateFormat}
 
 import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
@@ -122,9 +121,9 @@ case class Cast(child: Expression, dataType: DataType) 
extends UnaryExpression w
   // UDFToString
   private[this] def castToString(from: DataType): Any => Any = from match {
     case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes)
-    case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.toString(d)))
+    case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.dateToString(d)))
     case TimestampType => buildCast[Long](_,
-      t => 
UTF8String.fromString(timestampToString(DateTimeUtils.toJavaTimestamp(t))))
+      t => UTF8String.fromString(DateTimeUtils.timestampToString(t)))
     case _ => buildCast[Any](_, o => UTF8String.fromString(o.toString))
   }
 
@@ -183,7 +182,7 @@ case class Cast(child: Expression, dataType: DataType) 
extends UnaryExpression w
     case ByteType =>
       buildCast[Byte](_, b => longToTimestamp(b.toLong))
     case DateType =>
-      buildCast[Int](_, d => DateTimeUtils.toMillisSinceEpoch(d) * 10000)
+      buildCast[Int](_, d => DateTimeUtils.daysToMillis(d) * 10000)
     // TimestampWritable.decimalToTimestamp
     case DecimalType() =>
       buildCast[Decimal](_, d => decimalToTimestamp(d))
@@ -216,18 +215,6 @@ case class Cast(child: Expression, dataType: DataType) 
extends UnaryExpression w
     ts / 10000000.0
   }
 
-  // Converts Timestamp to string according to Hive TimestampWritable 
convention
-  private[this] def timestampToString(ts: Timestamp): String = {
-    val timestampString = ts.toString
-    val formatted = Cast.threadLocalTimestampFormat.get.format(ts)
-
-    if (timestampString.length > 19 && timestampString.substring(19) != ".0") {
-      formatted + timestampString.substring(19)
-    } else {
-      formatted
-    }
-  }
-
   // DateConverter
   private[this] def castToDate(from: DataType): Any => Any = from match {
     case StringType =>
@@ -449,11 +436,11 @@ case class Cast(child: Expression, dataType: DataType) 
extends UnaryExpression w
       case (DateType, StringType) =>
         defineCodeGen(ctx, ev, c =>
           s"""${ctx.stringType}.fromString(
-                
org.apache.spark.sql.catalyst.util.DateTimeUtils.toString($c))""")
-      // Special handling required for timestamps in hive test cases since the 
toString function
-      // does not match the expected output.
+                
org.apache.spark.sql.catalyst.util.DateTimeUtils.dateToString($c))""")
       case (TimestampType, StringType) =>
-        super.genCode(ctx, ev)
+        defineCodeGen(ctx, ev, c =>
+          s"""${ctx.stringType}.fromString(
+                
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c))""")
       case (_, StringType) =>
         defineCodeGen(ctx, ev, c => 
s"${ctx.stringType}.fromString(String.valueOf($c))")
 
@@ -477,19 +464,3 @@ case class Cast(child: Expression, dataType: DataType) 
extends UnaryExpression w
     }
   }
 }
-
-object Cast {
-  // `SimpleDateFormat` is not thread-safe.
-  private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] {
-    override def initialValue(): SimpleDateFormat = {
-      new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
-    }
-  }
-
-  // `SimpleDateFormat` is not thread-safe.
-  private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] {
-    override def initialValue(): SimpleDateFormat = {
-      new SimpleDateFormat("yyyy-MM-dd")
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/881662e9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
index ff79884..640e67e 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
@@ -18,11 +18,9 @@
 package org.apache.spark.sql.catalyst.util
 
 import java.sql.{Date, Timestamp}
-import java.text.SimpleDateFormat
+import java.text.{DateFormat, SimpleDateFormat}
 import java.util.{Calendar, TimeZone}
 
-import org.apache.spark.sql.catalyst.expressions.Cast
-
 /**
  * Helper functions for converting between internal and external date and time 
representations.
  * Dates are exposed externally as java.sql.Date and are represented 
internally as the number of
@@ -41,35 +39,53 @@ object DateTimeUtils {
 
 
   // Java TimeZone has no mention of thread safety. Use thread local instance 
to be safe.
-  private val LOCAL_TIMEZONE = new ThreadLocal[TimeZone] {
+  private val threadLocalLocalTimeZone = new ThreadLocal[TimeZone] {
     override protected def initialValue: TimeZone = {
       Calendar.getInstance.getTimeZone
     }
   }
 
-  private def javaDateToDays(d: Date): Int = {
-    millisToDays(d.getTime)
+  // `SimpleDateFormat` is not thread-safe.
+  private val threadLocalTimestampFormat = new ThreadLocal[DateFormat] {
+    override def initialValue(): SimpleDateFormat = {
+      new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
+    }
+  }
+
+  // `SimpleDateFormat` is not thread-safe.
+  private val threadLocalDateFormat = new ThreadLocal[DateFormat] {
+    override def initialValue(): SimpleDateFormat = {
+      new SimpleDateFormat("yyyy-MM-dd")
+    }
   }
 
+
   // we should use the exact day as Int, for example, (year, month, day) -> day
   def millisToDays(millisLocal: Long): Int = {
-    ((millisLocal + LOCAL_TIMEZONE.get().getOffset(millisLocal)) / 
MILLIS_PER_DAY).toInt
+    ((millisLocal + threadLocalLocalTimeZone.get().getOffset(millisLocal)) / 
MILLIS_PER_DAY).toInt
   }
 
-  def toMillisSinceEpoch(days: Int): Long = {
+  // reverse of millisToDays
+  def daysToMillis(days: Int): Long = {
     val millisUtc = days.toLong * MILLIS_PER_DAY
-    millisUtc - LOCAL_TIMEZONE.get().getOffset(millisUtc)
+    millisUtc - threadLocalLocalTimeZone.get().getOffset(millisUtc)
   }
 
-  def fromJavaDate(date: Date): Int = {
-    javaDateToDays(date)
-  }
+  def dateToString(days: Int): String =
+    threadLocalDateFormat.get.format(toJavaDate(days))
 
-  def toJavaDate(daysSinceEpoch: Int): Date = {
-    new Date(toMillisSinceEpoch(daysSinceEpoch))
-  }
+  // Converts Timestamp to string according to Hive TimestampWritable 
convention.
+  def timestampToString(num100ns: Long): String = {
+    val ts = toJavaTimestamp(num100ns)
+    val timestampString = ts.toString
+    val formatted = threadLocalTimestampFormat.get.format(ts)
 
-  def toString(days: Int): String = 
Cast.threadLocalDateFormat.get.format(toJavaDate(days))
+    if (timestampString.length > 19 && timestampString.substring(19) != ".0") {
+      formatted + timestampString.substring(19)
+    } else {
+      formatted
+    }
+  }
 
   def stringToTime(s: String): java.util.Date = {
     if (!s.contains('T')) {
@@ -100,7 +116,21 @@ object DateTimeUtils {
   }
 
   /**
-   * Return a java.sql.Timestamp from number of 100ns since epoch
+   * Returns the number of days since epoch from from java.sql.Date.
+   */
+  def fromJavaDate(date: Date): Int = {
+    millisToDays(date.getTime)
+  }
+
+  /**
+   * Returns a java.sql.Date from number of days since epoch.
+   */
+  def toJavaDate(daysSinceEpoch: Int): Date = {
+    new Date(daysToMillis(daysSinceEpoch))
+  }
+
+  /**
+   * Returns a java.sql.Timestamp from number of 100ns since epoch.
    */
   def toJavaTimestamp(num100ns: Long): Timestamp = {
     // setNanos() will overwrite the millisecond part, so the milliseconds 
should be
@@ -118,7 +148,7 @@ object DateTimeUtils {
   }
 
   /**
-   * Return the number of 100ns since epoch from java.sql.Timestamp.
+   * Returns the number of 100ns since epoch from java.sql.Timestamp.
    */
   def fromJavaTimestamp(t: Timestamp): Long = {
     if (t != null) {
@@ -129,7 +159,7 @@ object DateTimeUtils {
   }
 
   /**
-   * Return the number of 100ns (hundred of nanoseconds) since epoch from 
Julian day
+   * Returns the number of 100ns (hundred of nanoseconds) since epoch from 
Julian day
    * and nanoseconds in a day
    */
   def fromJulianDay(day: Int, nanoseconds: Long): Long = {
@@ -139,7 +169,7 @@ object DateTimeUtils {
   }
 
   /**
-   * Return Julian day and nanoseconds in a day from the number of 100ns 
(hundred of nanoseconds)
+   * Returns Julian day and nanoseconds in a day from the number of 100ns 
(hundred of nanoseconds)
    */
   def toJulianDay(num100ns: Long): (Int, Long) = {
     val seconds = num100ns / HUNDRED_NANOS_PER_SECOND + SECONDS_PER_DAY / 2

http://git-wip-us.apache.org/repos/asf/spark/blob/881662e9/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
index ab75b12..ecc78a5 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
@@ -201,7 +201,7 @@ private[spark] class 
SparkHiveDynamicPartitionWriterContainer(
     def convertToHiveRawString(col: String, value: Any): String = {
       val raw = String.valueOf(value)
       schema(col).dataType match {
-        case DateType => DateTimeUtils.toString(raw.toInt)
+        case DateType => DateTimeUtils.dateToString(raw.toInt)
         case _: DecimalType => BigDecimal(raw).toString()
         case _ => raw
       }


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

Reply via email to