[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-15 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241944517
 
 

 ##
 File path: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala
 ##
 @@ -125,56 +126,58 @@ abstract class HadoopFsRelationTest extends QueryTest 
with SQLTestUtils with Tes
 } else {
   Seq(false)
 }
-for (dataType <- supportedDataTypes) {
-  for (parquetDictionaryEncodingEnabled <- 
parquetDictionaryEncodingEnabledConfs) {
-val extraMessage = if (isParquetDataSource) {
-  s" with parquet.enable.dictionary = 
$parquetDictionaryEncodingEnabled"
-} else {
-  ""
-}
-logInfo(s"Testing $dataType data type$extraMessage")
-
-val extraOptions = Map[String, String](
-  "parquet.enable.dictionary" -> 
parquetDictionaryEncodingEnabled.toString
-)
-
-withTempPath { file =>
-  val path = file.getCanonicalPath
-
-  val dataGenerator = RandomDataGenerator.forType(
-dataType = dataType,
-nullable = true,
-new Random(System.nanoTime())
-  ).getOrElse {
-fail(s"Failed to create data generator for schema $dataType")
+withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") {
 
 Review comment:
   Here is the ticket: https://issues.apache.org/jira/browse/SPARK-26374 and I 
added TODO


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-14 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241817231
 
 

 ##
 File path: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala
 ##
 @@ -125,56 +126,58 @@ abstract class HadoopFsRelationTest extends QueryTest 
with SQLTestUtils with Tes
 } else {
   Seq(false)
 }
-for (dataType <- supportedDataTypes) {
-  for (parquetDictionaryEncodingEnabled <- 
parquetDictionaryEncodingEnabledConfs) {
-val extraMessage = if (isParquetDataSource) {
-  s" with parquet.enable.dictionary = 
$parquetDictionaryEncodingEnabled"
-} else {
-  ""
-}
-logInfo(s"Testing $dataType data type$extraMessage")
-
-val extraOptions = Map[String, String](
-  "parquet.enable.dictionary" -> 
parquetDictionaryEncodingEnabled.toString
-)
-
-withTempPath { file =>
-  val path = file.getCanonicalPath
-
-  val dataGenerator = RandomDataGenerator.forType(
-dataType = dataType,
-nullable = true,
-new Random(System.nanoTime())
-  ).getOrElse {
-fail(s"Failed to create data generator for schema $dataType")
+withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") {
 
 Review comment:
   What about to put the test under the flag 
`spark.sql.legacy.timeParser.enabled` and create a separate JIRA ticket? I 
would believe the bug somewhere in Spark's home made date/time functions rather 
than Java 8 implementation of timestamps parsing.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-14 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241809225
 
 

 ##
 File path: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala
 ##
 @@ -125,56 +126,58 @@ abstract class HadoopFsRelationTest extends QueryTest 
with SQLTestUtils with Tes
 } else {
   Seq(false)
 }
-for (dataType <- supportedDataTypes) {
-  for (parquetDictionaryEncodingEnabled <- 
parquetDictionaryEncodingEnabledConfs) {
-val extraMessage = if (isParquetDataSource) {
-  s" with parquet.enable.dictionary = 
$parquetDictionaryEncodingEnabled"
-} else {
-  ""
-}
-logInfo(s"Testing $dataType data type$extraMessage")
-
-val extraOptions = Map[String, String](
-  "parquet.enable.dictionary" -> 
parquetDictionaryEncodingEnabled.toString
-)
-
-withTempPath { file =>
-  val path = file.getCanonicalPath
-
-  val dataGenerator = RandomDataGenerator.forType(
-dataType = dataType,
-nullable = true,
-new Random(System.nanoTime())
-  ).getOrElse {
-fail(s"Failed to create data generator for schema $dataType")
+withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") {
 
 Review comment:
   > see which seed can reproduce the bug and debug it locally.
   
   I ran it locally many times. It is almost 100% reproducible for any seed.  


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-14 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241703085
 
 

 ##
 File path: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala
 ##
 @@ -125,56 +126,58 @@ abstract class HadoopFsRelationTest extends QueryTest 
with SQLTestUtils with Tes
 } else {
   Seq(false)
 }
-for (dataType <- supportedDataTypes) {
-  for (parquetDictionaryEncodingEnabled <- 
parquetDictionaryEncodingEnabledConfs) {
-val extraMessage = if (isParquetDataSource) {
-  s" with parquet.enable.dictionary = 
$parquetDictionaryEncodingEnabled"
-} else {
-  ""
-}
-logInfo(s"Testing $dataType data type$extraMessage")
-
-val extraOptions = Map[String, String](
-  "parquet.enable.dictionary" -> 
parquetDictionaryEncodingEnabled.toString
-)
-
-withTempPath { file =>
-  val path = file.getCanonicalPath
-
-  val dataGenerator = RandomDataGenerator.forType(
-dataType = dataType,
-nullable = true,
-new Random(System.nanoTime())
-  ).getOrElse {
-fail(s"Failed to create data generator for schema $dataType")
+withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") {
 
 Review comment:
   Not sure, this is related to 1970. I added a few round trip tests


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-13 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241480941
 
 

 ##
 File path: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala
 ##
 @@ -125,56 +126,58 @@ abstract class HadoopFsRelationTest extends QueryTest 
with SQLTestUtils with Tes
 } else {
   Seq(false)
 }
-for (dataType <- supportedDataTypes) {
-  for (parquetDictionaryEncodingEnabled <- 
parquetDictionaryEncodingEnabledConfs) {
-val extraMessage = if (isParquetDataSource) {
-  s" with parquet.enable.dictionary = 
$parquetDictionaryEncodingEnabled"
-} else {
-  ""
-}
-logInfo(s"Testing $dataType data type$extraMessage")
-
-val extraOptions = Map[String, String](
-  "parquet.enable.dictionary" -> 
parquetDictionaryEncodingEnabled.toString
-)
-
-withTempPath { file =>
-  val path = file.getCanonicalPath
-
-  val dataGenerator = RandomDataGenerator.forType(
-dataType = dataType,
-nullable = true,
-new Random(System.nanoTime())
-  ).getOrElse {
-fail(s"Failed to create data generator for schema $dataType")
+withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") {
 
 Review comment:
   Let me look at it deeper.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-13 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241414598
 
 

 ##
 File path: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala
 ##
 @@ -125,56 +126,58 @@ abstract class HadoopFsRelationTest extends QueryTest 
with SQLTestUtils with Tes
 } else {
   Seq(false)
 }
-for (dataType <- supportedDataTypes) {
-  for (parquetDictionaryEncodingEnabled <- 
parquetDictionaryEncodingEnabledConfs) {
-val extraMessage = if (isParquetDataSource) {
-  s" with parquet.enable.dictionary = 
$parquetDictionaryEncodingEnabled"
-} else {
-  ""
-}
-logInfo(s"Testing $dataType data type$extraMessage")
-
-val extraOptions = Map[String, String](
-  "parquet.enable.dictionary" -> 
parquetDictionaryEncodingEnabled.toString
-)
-
-withTempPath { file =>
-  val path = file.getCanonicalPath
-
-  val dataGenerator = RandomDataGenerator.forType(
-dataType = dataType,
-nullable = true,
-new Random(System.nanoTime())
-  ).getOrElse {
-fail(s"Failed to create data generator for schema $dataType")
+withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") {
 
 Review comment:
   > I'm a little worried here. This test is a round-trip test ...
   > It should be same, if the session local timezone doesn't change between 
write and read back.
   
   Not only JSON parser/formatter involved in the loop but also converting 
milliseconds to Java's `Timestamp` and to something else.
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-13 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241383509
 
 

 ##
 File path: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala
 ##
 @@ -125,56 +126,58 @@ abstract class HadoopFsRelationTest extends QueryTest 
with SQLTestUtils with Tes
 } else {
   Seq(false)
 }
-for (dataType <- supportedDataTypes) {
-  for (parquetDictionaryEncodingEnabled <- 
parquetDictionaryEncodingEnabledConfs) {
-val extraMessage = if (isParquetDataSource) {
-  s" with parquet.enable.dictionary = 
$parquetDictionaryEncodingEnabled"
-} else {
-  ""
-}
-logInfo(s"Testing $dataType data type$extraMessage")
-
-val extraOptions = Map[String, String](
-  "parquet.enable.dictionary" -> 
parquetDictionaryEncodingEnabled.toString
-)
-
-withTempPath { file =>
-  val path = file.getCanonicalPath
-
-  val dataGenerator = RandomDataGenerator.forType(
-dataType = dataType,
-nullable = true,
-new Random(System.nanoTime())
-  ).getOrElse {
-fail(s"Failed to create data generator for schema $dataType")
+withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") {
 
 Review comment:
   `RandomDataGenerator` produces timestamps in `UTC` time zone, and Spark's 
settings for time zones are not taken into account as far as I remember.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-13 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241349476
 
 

 ##
 File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
 ##
 @@ -96,19 +99,26 @@ class JsonSuite extends QueryTest with SharedSQLContext 
with TestJsonData {
 checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new 
Timestamp(intNumber.toLong * 1000L)),
 enforceCorrectType(intNumber.toLong, TimestampType))
 val strTime = "2014-09-30 12:34:56"
-
checkTypePromotion(DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(strTime)),
-enforceCorrectType(strTime, TimestampType))
+checkTypePromotion(
+  expected = 141208049600L,
 
 Review comment:
   As far as I remember `"GMT"` is passed as default time zone to JSONOptions 
https://github.com/apache/spark/pull/23196/files#diff-fde14032b0e6ef8086461edf79a27c5dL67
 and this guy `DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(strTime)` does 
not care about any Spark settings and takes system timezone from jvm.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-13 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241317291
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatter.scala
 ##
 @@ -33,26 +33,37 @@ sealed trait DateTimeFormatter {
   def format(us: Long): String
 }
 
+trait FormatterUtils {
+  def zoneId: ZoneId
+  def buildFormatter(pattern: String, locale: Locale): 
java.time.format.DateTimeFormatter = {
+new DateTimeFormatterBuilder()
+  .appendPattern(pattern)
+  .parseDefaulting(ChronoField.YEAR_OF_ERA, 1970)
+  .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1)
+  .parseDefaulting(ChronoField.DAY_OF_MONTH, 1)
+  .parseDefaulting(ChronoField.HOUR_OF_DAY, 0)
+  .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0)
+  .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0)
+  .toFormatter(locale)
+  }
+  def toInstant(temporalAccessor: TemporalAccessor): java.time.Instant = {
 
 Review comment:
   Strictly speaking it uses non-default time zone, what about 
`toInstantWithZoneId`?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-13 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241316473
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatter.scala
 ##
 @@ -116,29 +127,32 @@ sealed trait DateFormatter {
 
 class Iso8601DateFormatter(
 pattern: String,
-timeZone: TimeZone,
-locale: Locale) extends DateFormatter {
+locale: Locale) extends DateFormatter with FormatterUtils {
 
-  val dateTimeFormatter = new Iso8601DateTimeFormatter(pattern, timeZone, 
locale)
+  val zoneId = ZoneId.of("GMT")
 
 Review comment:
   Thanks for the link. I probably read the same when I thought what is better 
to use. What confused me is **GMT is a time zone** vs **UTC is not a time zone**
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-13 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241315581
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatter.scala
 ##
 @@ -33,26 +33,37 @@ sealed trait DateTimeFormatter {
   def format(us: Long): String
 }
 
+trait FormatterUtils {
+  def zoneId: ZoneId
+  def buildFormatter(pattern: String, locale: Locale): 
java.time.format.DateTimeFormatter = {
+new DateTimeFormatterBuilder()
+  .appendPattern(pattern)
+  .parseDefaulting(ChronoField.YEAR_OF_ERA, 1970)
+  .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1)
+  .parseDefaulting(ChronoField.DAY_OF_MONTH, 1)
+  .parseDefaulting(ChronoField.HOUR_OF_DAY, 0)
+  .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0)
+  .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0)
+  .toFormatter(locale)
+  }
+  def toInstant(temporalAccessor: TemporalAccessor): java.time.Instant = {
+val localDateTime = LocalDateTime.from(temporalAccessor)
+val zonedDateTime = ZonedDateTime.of(localDateTime, zoneId)
+Instant.from(zonedDateTime)
+  }
+}
+
 class Iso8601DateTimeFormatter(
 pattern: String,
 timeZone: TimeZone,
-locale: Locale) extends DateTimeFormatter {
-  val formatter = new DateTimeFormatterBuilder()
-.appendPattern(pattern)
-.parseDefaulting(ChronoField.YEAR_OF_ERA, 1970)
-.parseDefaulting(ChronoField.MONTH_OF_YEAR, 1)
-.parseDefaulting(ChronoField.DAY_OF_MONTH, 1)
-.parseDefaulting(ChronoField.HOUR_OF_DAY, 0)
-.parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0)
-.parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0)
-.toFormatter(locale)
+locale: Locale) extends DateTimeFormatter with FormatterUtils {
+  val zoneId = timeZone.toZoneId
+  val formatter = buildFormatter(pattern, locale)
 
   def toInstant(s: String): Instant = {
 val temporalAccessor = formatter.parse(s)
 if (temporalAccessor.query(TemporalQueries.offset()) == null) {
 
 Review comment:
   zone offset is unknown after parsing. For example, if you parse `13-12-2018 
09:55:00`, it is unclear in which timezone it is.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-13 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241314965
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatter.scala
 ##
 @@ -33,26 +33,37 @@ sealed trait DateTimeFormatter {
   def format(us: Long): String
 }
 
+trait FormatterUtils {
+  def zoneId: ZoneId
+  def buildFormatter(pattern: String, locale: Locale): 
java.time.format.DateTimeFormatter = {
+new DateTimeFormatterBuilder()
+  .appendPattern(pattern)
+  .parseDefaulting(ChronoField.YEAR_OF_ERA, 1970)
+  .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1)
+  .parseDefaulting(ChronoField.DAY_OF_MONTH, 1)
+  .parseDefaulting(ChronoField.HOUR_OF_DAY, 0)
+  .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0)
+  .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0)
 
 Review comment:
   Let's say ff you need to parse `Dec 2018`, the parser doesn't know what day 
it should use by default.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON

2018-12-13 Thread GitBox
MaxGekk commented on a change in pull request #23196: [SPARK-26243][SQL] Use 
java.time API for parsing timestamps and dates from JSON
URL: https://github.com/apache/spark/pull/23196#discussion_r241314004
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
 ##
 @@ -77,6 +77,12 @@ private[sql] class JacksonGenerator(
 
   private val lineSeparator: String = options.lineSeparatorInWrite
 
+  private val timeFormatter = DateTimeFormatter(
 
 Review comment:
   `Date`/`TimestampFormatter` ?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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