You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/06/09 16:41:17 UTC

[GitHub] [spark] yaooqinn opened a new pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

yaooqinn opened a new pull request #28766:
URL: https://github.com/apache/spark/pull/28766


   ### What changes were proposed in this pull request?
   
   If a datetime pattern contains no year field, the day of year field should not be ignored if exists
   
   e.g.
   
   ```
   spark-sql> select to_timestamp('31', 'DD');
   1970-01-01 00:00:00
   spark-sql> select to_timestamp('31 30', 'DD dd');
   1970-01-30 00:00:00
   
   spark.sql.legacy.timeParserPolicy legacy
   spark-sql> select to_timestamp('31', 'DD');
   1970-01-31 00:00:00
   spark-sql> select to_timestamp('31 30', 'DD dd');
   NULL
   ```
   
   This PR only fixes some corner cases that use 'D' pattern to parse datetimes and there is w/o 'y'.
   
   ### Why are the changes needed?
   
   fix some corner cases
   
   
   ### Does this PR introduce _any_ user-facing change?
   
   yes, the day of year field will not be ignored 
   
   
   ### How was this patch tested?
   
   
   add unit tests.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan closed pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #28766:
URL: https://github.com/apache/spark/pull/28766


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641806950


   **[Test build #123741 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123741/testReport)** for PR 28766 at commit [`cd55187`](https://github.com/apache/spark/commit/cd55187949560be6df78c268e1d9695adfb6d90b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641998322






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642049557


   **[Test build #123768 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123768/testReport)** for PR 28766 at commit [`78c676e`](https://github.com/apache/spark/commit/78c676e1d60a5c9c785b40d3b42ac756b72c54e4).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641998322


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642223510


   **[Test build #123768 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123768/testReport)** for PR 28766 at commit [`78c676e`](https://github.com/apache/spark/commit/78c676e1d60a5c9c785b40d3b42ac756b72c54e4).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641818488






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437960964



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
##########
@@ -21,11 +21,44 @@ import org.scalatest.Matchers
 
 import org.apache.spark.{SparkFunSuite, SparkUpgradeException}
 import org.apache.spark.sql.catalyst.plans.SQLHelper
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{date, UTC}
 
 trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers {
   import DateTimeFormatterHelper._
   def checkFormatterCreation(pattern: String, isParsing: Boolean): Unit
 
+  private def dateFormatter(pattern: String): DateFormatter = {
+    DateFormatter(pattern, UTC, isParsing = true)
+  }
+
+  private def timestampFormatter(pattern: String): TimestampFormatter =
+    TimestampFormatter(pattern, UTC, isParsing = true)
+
+  protected def useDateFormatter: Boolean
+
+  private def assertEqual(pattern: String, datetimeStr: String, expected: Long): Unit = {
+    if (useDateFormatter) {
+      assert(dateFormatter(pattern).parse(datetimeStr) ===
+        DateTimeUtils.microsToEpochDays(expected, UTC))
+    } else {
+      assert(timestampFormatter(pattern).parse(datetimeStr) === expected)
+    }
+  }
+
+  private def assertError(pattern: String, datetimeStr: String, expectedMsg: String): Unit = {
+    val exception = if (useDateFormatter) {
+      intercept[Exception](dateFormatter(pattern).parse(datetimeStr))

Review comment:
       Ok, for the current test cases, we can capture only `SparkUpgradeException` here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642189380


   Merged build finished. Test PASSed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r438146490



##########
File path: sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql
##########
@@ -0,0 +1,17 @@
+--- TESTS FOR DATETIME PARSING FUNCTIONS ---
+
+-- parsing with pattern 'D'
+select to_timestamp('9', 'D');
+select to_timestamp('300', 'D');
+select to_timestamp('09', 'DD');
+select to_timestamp('99', 'DD');
+select to_timestamp('009', 'DDD');
+select to_timestamp('365', 'DDD');
+select to_timestamp('31-365', 'dd-DDD');
+select to_timestamp('12-365', 'MM-DDD');
+select to_timestamp('2020-365', 'yyyy-DDD');
+select to_timestamp('12-31-365', 'MM-dd-DDD');
+select to_timestamp('2020-30-365', 'yyyy-dd-DDD');
+select to_timestamp('2020-12-350', 'yyyy-MM-DDD');
+select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD');
+select from_csv('2018-365', 'date Date', map('dateFormat', 'yyyy-DDD'))

Review comment:
       for non-error case, csv has no difference, we don't need to test it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642073047


   **[Test build #123747 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123747/testReport)** for PR 28766 at commit [`b3cdccf`](https://github.com/apache/spark/commit/b3cdccf1fb9ac083b7442866f9adad63236e8067).
    * This patch **fails PySpark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642074451






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437834431



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala
##########
@@ -39,6 +39,18 @@ trait DateTimeFormatterHelper {
     }
   }
 
+  private def verifyLocalDate(
+      accessor: TemporalAccessor, field: ChronoField, candidate: LocalDate): Unit = {
+    if (accessor.isSupported(field) && candidate.isSupported(field)) {
+      val actual = accessor.get(field)
+      val expected = candidate.get(field)
+      if (actual != expected) {
+        throw new DateTimeException(s"Conflict found: Field $field $actual differs from" +
+          s" $field $expected derived from $candidate")

Review comment:
       can you show an example of this error message? let's see if it looks good.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r438145481



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
##########
@@ -17,15 +17,62 @@
 
 package org.apache.spark.sql.catalyst.util
 
+import java.time.DateTimeException
+
 import org.scalatest.Matchers
 
 import org.apache.spark.{SparkFunSuite, SparkUpgradeException}
 import org.apache.spark.sql.catalyst.plans.SQLHelper
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{date, UTC}
 
 trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers {
   import DateTimeFormatterHelper._
+  import LegacyDateFormats._
   def checkFormatterCreation(pattern: String, isParsing: Boolean): Unit
 
+  private def dateFormatter(
+      pattern: String,
+      ldf: LegacyDateFormat = FAST_DATE_FORMAT): DateFormatter = {

Review comment:
       super nit: we can put the parameters in one line




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641992956


   **[Test build #123741 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123741/testReport)** for PR 28766 at commit [`cd55187`](https://github.com/apache/spark/commit/cd55187949560be6df78c268e1d9695adfb6d90b).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642101060


   **[Test build #123743 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123743/testReport)** for PR 28766 at commit [`20b585d`](https://github.com/apache/spark/commit/20b585d411c434140139ba3ca0fe3ea516a23b6b).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437966022



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
##########
@@ -21,11 +21,39 @@ import org.scalatest.Matchers
 
 import org.apache.spark.{SparkFunSuite, SparkUpgradeException}
 import org.apache.spark.sql.catalyst.plans.SQLHelper
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{date, UTC}
 
 trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers {
   import DateTimeFormatterHelper._
   def checkFormatterCreation(pattern: String, isParsing: Boolean): Unit
 
+  private def dateFormatter(pattern: String): DateFormatter = {
+    DateFormatter(pattern, UTC, isParsing = true)
+  }
+
+  private def timestampFormatter(pattern: String): TimestampFormatter =
+    TimestampFormatter(pattern, UTC, isParsing = true)
+
+  protected def useDateFormatter: Boolean
+
+  private def assertEqual(pattern: String, datetimeStr: String, expected: Long): Unit = {
+    if (useDateFormatter) {
+      assert(dateFormatter(pattern).parse(datetimeStr) ===
+        DateTimeUtils.microsToEpochDays(expected, UTC))
+    } else {
+      assert(timestampFormatter(pattern).parse(datetimeStr) === expected)
+    }
+  }
+
+  private def assertError(pattern: String, datetimeStr: String, expectedMsg: String): Unit = {
+    val exception = if (useDateFormatter) {
+      intercept[SparkUpgradeException](dateFormatter(pattern).parse(datetimeStr))

Review comment:
       does this mean the legacy formatter won't fail?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642049557


   **[Test build #123768 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123768/testReport)** for PR 28766 at commit [`78c676e`](https://github.com/apache/spark/commit/78c676e1d60a5c9c785b40d3b42ac756b72c54e4).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon edited a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
HyukjinKwon edited a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642671940


   I reverted this from 3.0 - it causes test failure. @yaooqinn can you create a PR to backport this to branch-3.0?
   
   https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123839/testReport/


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641995324


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642103650






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642045959






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641879738


   **[Test build #123753 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123753/testReport)** for PR 28766 at commit [`6b8dec4`](https://github.com/apache/spark/commit/6b8dec4b8905fac295ac978a344a5a15eb9fb90a).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641851715


   **[Test build #123747 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123747/testReport)** for PR 28766 at commit [`b3cdccf`](https://github.com/apache/spark/commit/b3cdccf1fb9ac083b7442866f9adad63236e8067).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642103650






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437834522



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
##########
@@ -433,4 +433,35 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite {
       assert(formatter.format(date(1970, 4, 10)) == "100")
     }
   }
+
+  test("SPARK-31939: Fix Parsing day of year when year field pattern is missing") {
+    // resolved to queryable LocaleDate or fail directly
+    val f0 = TimestampFormatter("yyyy-dd-DD", UTC, isParsing = true)
+    assert(f0.parse("2020-29-60") === date(2020, 2, 29))
+    assertParsingError(f0.parse("2020-02-60"))
+    val f1 = TimestampFormatter("yyyy-MM-DD", UTC, isParsing = true)
+    assert(f1.parse("2020-02-60") === date(2020, 2, 29))
+    assertParsingError(f1.parse("2020-03-60"))
+    val f2 = TimestampFormatter("yyyy-MM-dd-DD", UTC, isParsing = true)
+    assert(f2.parse("2020-02-29-60") === date(2020, 2, 29))
+    assertParsingError(f2.parse("2020-03-01-60"))
+    val f3 = TimestampFormatter("yyyy-DDD", UTC, isParsing = true)
+    assert(f3.parse("2020-366") === date(2020, 12, 31))
+    assertParsingError(f3.parse("2019-366"))
+
+    // unresolved and need to check manually(SPARK-31939 fixed)
+    val f4 = TimestampFormatter("DDD", UTC, isParsing = true)
+    assert(f4.parse("365") === date(1970, 12, 31))
+    assertParsingError(f4.parse("366")) // 1970 is not a leap year
+    val f5 = TimestampFormatter("MM-DD", UTC, isParsing = true)
+    assert(f5.parse("03-60") === date(1970, 3, 1))
+    assertParsingError(f5.parse("02-60"))
+    val f6 = TimestampFormatter("MM-dd-DD", UTC, isParsing = true)
+    assert(f6.parse("02-28-59") === date(1970, 2, 28))
+    assertParsingError(f6.parse("02-28-60"))
+    assertParsingError(f6.parse("02-28-58"))
+    val f7 = TimestampFormatter("dd-DD", UTC, isParsing = true)
+    assert(f7.parse("28-59") === date(1970, 2, 28))
+    assertParsingError(f7.parse("27-59"))
+  }

Review comment:
       can we also add tests in the .sql file?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641808553






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437966472



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
##########
@@ -51,4 +79,36 @@ trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
       pattern => intercept[SparkUpgradeException](checkFormatterCreation(pattern, true))
     }
   }
+
+  test("SPARK-31939: Fix Parsing day of year when year field pattern is missing") {
+    // resolved to queryable LocaleDate or fail directly
+    assertEqual("yyyy-dd-DD", "2020-29-60", date(2020, 2, 29))
+    assertError("yyyy-dd-DD", "2020-02-60",
+      "Field DayOfMonth 29 differs from DayOfMonth 2 derived from 2020-02-29")

Review comment:
       shall we remove `derived from 2020-02-29`? looks confusing.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437871769



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
##########
@@ -433,4 +433,35 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite {
       assert(formatter.format(date(1970, 4, 10)) == "100")
     }
   }
+
+  test("SPARK-31939: Fix Parsing day of year when year field pattern is missing") {
+    // resolved to queryable LocaleDate or fail directly
+    val f0 = TimestampFormatter("yyyy-dd-DD", UTC, isParsing = true)

Review comment:
       Sounds good.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437971312



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
##########
@@ -51,4 +79,36 @@ trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
       pattern => intercept[SparkUpgradeException](checkFormatterCreation(pattern, true))
     }
   }
+
+  test("SPARK-31939: Fix Parsing day of year when year field pattern is missing") {
+    // resolved to queryable LocaleDate or fail directly
+    assertEqual("yyyy-dd-DD", "2020-29-60", date(2020, 2, 29))
+    assertError("yyyy-dd-DD", "2020-02-60",
+      "Field DayOfMonth 29 differs from DayOfMonth 2 derived from 2020-02-29")

Review comment:
       
   > shall we remove `derived from 2020-02-29`? looks confusing.
   remove it from the test cases here or the original error? It's ok to remove it here, but cannot be done from the original error because it is thrown by java time lib.
   
   > what's the error message if we run the jdk time lib directly?
   
   `Text '2020-02-60' could not be parsed: Conflict found: Field DayOfMonth 29 differs from DayOfMonth 2 derived from 2020-02-29` is thrown by JDK. The DatetimeException we defined just omits the leading `Text '2020-02-60' could not be parsed: `
   
   
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642188154


   **[Test build #123763 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123763/testReport)** for PR 28766 at commit [`1899e81`](https://github.com/apache/spark/commit/1899e8149bc39583a9b0ccb17027eb0f809b8022).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641146441






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641995852


   **[Test build #123763 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123763/testReport)** for PR 28766 at commit [`1899e81`](https://github.com/apache/spark/commit/1899e8149bc39583a9b0ccb17027eb0f809b8022).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641818488






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641808553






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642149970






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641919836


   **[Test build #123757 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123757/testReport)** for PR 28766 at commit [`04ce7ee`](https://github.com/apache/spark/commit/04ce7ee66abb7a4d8772ab5779bfada28d9fb52d).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642149970






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] dongjoon-hyun commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642773405


   Thank you, @HyukjinKwon !


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641997335


   **[Test build #123753 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123753/testReport)** for PR 28766 at commit [`6b8dec4`](https://github.com/apache/spark/commit/6b8dec4b8905fac295ac978a344a5a15eb9fb90a).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641875140






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641920620






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641145651


   **[Test build #123683 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123683/testReport)** for PR 28766 at commit [`2d797e0`](https://github.com/apache/spark/commit/2d797e0410e9bb6293d957e125fb25e75ff9aecf).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641771608






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641680135






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642074451


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641919836


   **[Test build #123757 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123757/testReport)** for PR 28766 at commit [`04ce7ee`](https://github.com/apache/spark/commit/04ce7ee66abb7a4d8772ab5779bfada28d9fb52d).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641770461


   **[Test build #123715 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123715/testReport)** for PR 28766 at commit [`a11a049`](https://github.com/apache/spark/commit/a11a049ad735ea4375e1b742c2fd9ba0093248c8).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642225214






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641852273






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641678621






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641680135






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641997346






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437975237



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
##########
@@ -51,4 +79,36 @@ trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
       pattern => intercept[SparkUpgradeException](checkFormatterCreation(pattern, true))
     }
   }
+
+  test("SPARK-31939: Fix Parsing day of year when year field pattern is missing") {
+    // resolved to queryable LocaleDate or fail directly
+    assertEqual("yyyy-dd-DD", "2020-29-60", date(2020, 2, 29))
+    assertError("yyyy-dd-DD", "2020-02-60",
+      "Field DayOfMonth 29 differs from DayOfMonth 2 derived from 2020-02-29")

Review comment:
       OK then it's fine.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641920620






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642225214






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641679788


   **[Test build #123715 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123715/testReport)** for PR 28766 at commit [`a11a049`](https://github.com/apache/spark/commit/a11a049ad735ea4375e1b742c2fd9ba0093248c8).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437944533



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
##########
@@ -21,11 +21,44 @@ import org.scalatest.Matchers
 
 import org.apache.spark.{SparkFunSuite, SparkUpgradeException}
 import org.apache.spark.sql.catalyst.plans.SQLHelper
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{date, UTC}
 
 trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers {
   import DateTimeFormatterHelper._
   def checkFormatterCreation(pattern: String, isParsing: Boolean): Unit
 
+  private def dateFormatter(pattern: String): DateFormatter = {
+    DateFormatter(pattern, UTC, isParsing = true)
+  }
+
+  private def timestampFormatter(pattern: String): TimestampFormatter =
+    TimestampFormatter(pattern, UTC, isParsing = true)
+
+  protected def useDateFormatter: Boolean
+
+  private def assertEqual(pattern: String, datetimeStr: String, expected: Long): Unit = {
+    if (useDateFormatter) {
+      assert(dateFormatter(pattern).parse(datetimeStr) ===
+        DateTimeUtils.microsToEpochDays(expected, UTC))
+    } else {
+      assert(timestampFormatter(pattern).parse(datetimeStr) === expected)
+    }
+  }
+
+  private def assertError(pattern: String, datetimeStr: String, expectedMsg: String): Unit = {
+    val exception = if (useDateFormatter) {
+      intercept[Exception](dateFormatter(pattern).parse(datetimeStr))

Review comment:
       can we specify the exception class?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642045959






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641806950


   **[Test build #123741 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123741/testReport)** for PR 28766 at commit [`cd55187`](https://github.com/apache/spark/commit/cd55187949560be6df78c268e1d9695adfb6d90b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642148668


   **[Test build #123757 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123757/testReport)** for PR 28766 at commit [`04ce7ee`](https://github.com/apache/spark/commit/04ce7ee66abb7a4d8772ab5779bfada28d9fb52d).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn removed a comment on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
yaooqinn removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641678629


   cc @cloud-fan @maropu @MaxGekk please review this PR, thanks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641771608






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642695042


   checking on that. @HyukjinKwon, thanks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437833606



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala
##########
@@ -39,6 +39,18 @@ trait DateTimeFormatterHelper {
     }
   }
 
+  private def verifyLocalDate(
+      accessor: TemporalAccessor, field: ChronoField, candidate: LocalDate): Unit = {
+    if (accessor.isSupported(field) && candidate.isSupported(field)) {

Review comment:
       `candidate.isSupported(field)` this is always true?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642671940


   I reverted this from 3.0 - it causes test failure. @yaooqinn can you create a PR to backport this to branch-3.0?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437971312



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
##########
@@ -51,4 +79,36 @@ trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
       pattern => intercept[SparkUpgradeException](checkFormatterCreation(pattern, true))
     }
   }
+
+  test("SPARK-31939: Fix Parsing day of year when year field pattern is missing") {
+    // resolved to queryable LocaleDate or fail directly
+    assertEqual("yyyy-dd-DD", "2020-29-60", date(2020, 2, 29))
+    assertError("yyyy-dd-DD", "2020-02-60",
+      "Field DayOfMonth 29 differs from DayOfMonth 2 derived from 2020-02-29")

Review comment:
       > shall we remove `derived from 2020-02-29`? looks confusing.
   
   remove it from the test cases here or the original error? It's ok to remove it here, but cannot be done from the original error because it is thrown by java time lib.
   
   > what's the error message if we run the jdk time lib directly?
   
   `Text '2020-02-60' could not be parsed: Conflict found: Field DayOfMonth 29 differs from DayOfMonth 2 derived from 2020-02-29` is thrown by JDK. The DatetimeException we defined just omits the leading `Text '2020-02-60' could not be parsed: `
   
   
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641145651






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641851715


   **[Test build #123747 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123747/testReport)** for PR 28766 at commit [`b3cdccf`](https://github.com/apache/spark/commit/b3cdccf1fb9ac083b7442866f9adad63236e8067).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641879738


   **[Test build #123753 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123753/testReport)** for PR 28766 at commit [`6b8dec4`](https://github.com/apache/spark/commit/6b8dec4b8905fac295ac978a344a5a15eb9fb90a).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642074468


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/123747/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437945021



##########
File path: sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql
##########
@@ -0,0 +1,16 @@
+--- TESTS FOR DATETIME PARSING FUNCTIONS ---

Review comment:
       there are a lot more tests should be added here later :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437982608



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
##########
@@ -21,11 +21,39 @@ import org.scalatest.Matchers
 
 import org.apache.spark.{SparkFunSuite, SparkUpgradeException}
 import org.apache.spark.sql.catalyst.plans.SQLHelper
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{date, UTC}
 
 trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers {
   import DateTimeFormatterHelper._
   def checkFormatterCreation(pattern: String, isParsing: Boolean): Unit
 
+  private def dateFormatter(pattern: String): DateFormatter = {
+    DateFormatter(pattern, UTC, isParsing = true)
+  }
+
+  private def timestampFormatter(pattern: String): TimestampFormatter =
+    TimestampFormatter(pattern, UTC, isParsing = true)
+
+  protected def useDateFormatter: Boolean
+
+  private def assertEqual(pattern: String, datetimeStr: String, expected: Long): Unit = {
+    if (useDateFormatter) {
+      assert(dateFormatter(pattern).parse(datetimeStr) ===
+        DateTimeUtils.microsToEpochDays(expected, UTC))
+    } else {
+      assert(timestampFormatter(pattern).parse(datetimeStr) === expected)
+    }
+  }
+
+  private def assertError(pattern: String, datetimeStr: String, expectedMsg: String): Unit = {
+    val exception = if (useDateFormatter) {
+      intercept[SparkUpgradeException](dateFormatter(pattern).parse(datetimeStr))

Review comment:
       the legacy fomatter only fails when `datetimeStr ` contains non-digits which we don't need to test here




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641817999


   **[Test build #123743 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123743/testReport)** for PR 28766 at commit [`20b585d`](https://github.com/apache/spark/commit/20b585d411c434140139ba3ca0fe3ea516a23b6b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641997346






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641995324






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641679788


   **[Test build #123715 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123715/testReport)** for PR 28766 at commit [`a11a049`](https://github.com/apache/spark/commit/a11a049ad735ea4375e1b742c2fd9ba0093248c8).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641852273






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641817999


   **[Test build #123743 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123743/testReport)** for PR 28766 at commit [`20b585d`](https://github.com/apache/spark/commit/20b585d411c434140139ba3ca0fe3ea516a23b6b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641146441






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437833956



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
##########
@@ -433,4 +433,35 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite {
       assert(formatter.format(date(1970, 4, 10)) == "100")
     }
   }
+
+  test("SPARK-31939: Fix Parsing day of year when year field pattern is missing") {
+    // resolved to queryable LocaleDate or fail directly
+    val f0 = TimestampFormatter("yyyy-dd-DD", UTC, isParsing = true)

Review comment:
       shall we test both timestamp and date formatter? We can put the test in the base class.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641995852


   **[Test build #123763 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123763/testReport)** for PR 28766 at commit [`1899e81`](https://github.com/apache/spark/commit/1899e8149bc39583a9b0ccb17027eb0f809b8022).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641998348


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/123753/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641875140






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642189380


   Merged build finished. Test PASSed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437966637



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
##########
@@ -51,4 +79,36 @@ trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
       pattern => intercept[SparkUpgradeException](checkFormatterCreation(pattern, true))
     }
   }
+
+  test("SPARK-31939: Fix Parsing day of year when year field pattern is missing") {
+    // resolved to queryable LocaleDate or fail directly
+    assertEqual("yyyy-dd-DD", "2020-29-60", date(2020, 2, 29))
+    assertError("yyyy-dd-DD", "2020-02-60",
+      "Field DayOfMonth 29 differs from DayOfMonth 2 derived from 2020-02-29")

Review comment:
       what's the error message if we run the jdk time lib directly?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r438146215



##########
File path: sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql
##########
@@ -0,0 +1,18 @@
+--- TESTS FOR DATETIME PARSING FUNCTIONS WITH INVALID VALUES ---
+
+-- parsing invalid values with pattern 'D'
+select to_timestamp('366', 'D');
+select to_timestamp('9', 'DD');
+-- in java 8 this case is invalid, but valid in java 11, disabled for jenkins
+-- select to_timestamp('100', 'DD');
+select to_timestamp('366', 'DD');
+select to_timestamp('9', 'DDD');
+select to_timestamp('99', 'DDD');
+select to_timestamp('30-365', 'dd-DDD');
+select to_timestamp('11-365', 'MM-DDD');
+select to_timestamp('2019-366', 'yyyy-DDD');
+select to_timestamp('12-30-365', 'MM-dd-DDD');
+select to_timestamp('2020-01-365', 'yyyy-dd-DDD');
+select to_timestamp('2020-10-350', 'yyyy-MM-DDD');
+select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD');
+select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD'))

Review comment:
       let's add a comment to explain why we need to test csv: because it's lenient and Spark should throw upgrade exception.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #28766: [SPARK-31939][SQL] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28766:
URL: https://github.com/apache/spark/pull/28766#discussion_r437871990



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala
##########
@@ -39,6 +39,18 @@ trait DateTimeFormatterHelper {
     }
   }
 
+  private def verifyLocalDate(
+      accessor: TemporalAccessor, field: ChronoField, candidate: LocalDate): Unit = {
+    if (accessor.isSupported(field) && candidate.isSupported(field)) {

Review comment:
       For the time being, yes. I can remove this condition

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala
##########
@@ -39,6 +39,18 @@ trait DateTimeFormatterHelper {
     }
   }
 
+  private def verifyLocalDate(
+      accessor: TemporalAccessor, field: ChronoField, candidate: LocalDate): Unit = {
+    if (accessor.isSupported(field) && candidate.isSupported(field)) {
+      val actual = accessor.get(field)
+      val expected = candidate.get(field)
+      if (actual != expected) {
+        throw new DateTimeException(s"Conflict found: Field $field $actual differs from" +
+          s" $field $expected derived from $candidate")

Review comment:
       OK




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-641995333


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/123741/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on pull request #28766: [SPARK-31939][SQL][test-java11] Fix Parsing day of year when year field pattern is missing

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #28766:
URL: https://github.com/apache/spark/pull/28766#issuecomment-642384047


   thanks, merging to master/3.0!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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