Skip to content

Commit

Permalink
json data timestamp ISO8601 support
Browse files Browse the repository at this point in the history
  • Loading branch information
adrian-wang committed Nov 7, 2014
1 parent 6e9ef10 commit 50df6e7
Show file tree
Hide file tree
Showing 3 changed files with 40 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.sql.json

import org.apache.spark.sql.catalyst.types.decimal.Decimal
import org.apache.spark.sql.types.util.DataTypeConversions

import scala.collection.Map
import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper}
Expand Down Expand Up @@ -378,15 +379,15 @@ private[sql] object JsonRDD extends Logging {
private def toDate(value: Any): Date = {
value match {
// only support string as date
case value: java.lang.String => Date.valueOf(value)
case value: java.lang.String => new Date(DataTypeConversions.stringToTime(value).getTime)
}
}

private def toTimestamp(value: Any): Timestamp = {
value match {
case value: java.lang.Integer => new Timestamp(value.asInstanceOf[Int].toLong)
case value: java.lang.Long => new Timestamp(value)
case value: java.lang.String => Timestamp.valueOf(value)
case value: java.lang.String => toTimestamp(DataTypeConversions.stringToTime(value).getTime)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.sql.types.util

import java.text.SimpleDateFormat

import scala.collection.JavaConverters._

import org.apache.spark.sql._
Expand Down Expand Up @@ -129,6 +131,34 @@ protected[sql] object DataTypeConversions {
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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,13 @@ class JsonSuite extends QueryTest {

val strDate = "2014-10-15"
checkTypePromotion(Date.valueOf(strDate), enforceCorrectType(strDate, DateType))

val ISO8601Time1 = "1970-01-01T01:00:01.0Z"
checkTypePromotion(new Timestamp(3601000), enforceCorrectType(ISO8601Time1, TimestampType))
checkTypePromotion(new Date(3601000), enforceCorrectType(ISO8601Time1, DateType))
val ISO8601Time2 = "1970-01-01T02:00:01-01:00"
checkTypePromotion(new Timestamp(10801000), enforceCorrectType(ISO8601Time2, TimestampType))
checkTypePromotion(new Date(10801000), enforceCorrectType(ISO8601Time2, DateType))
}

test("Get compatible type") {
Expand Down

0 comments on commit 50df6e7

Please sign in to comment.