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/11/26 12:34:38 UTC

[GitHub] [spark] waitinfuture opened a new pull request #30516: [SPARK-33498][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

waitinfuture opened a new pull request #30516:
URL: https://github.com/apache/spark/pull/30516


   #29876 # What changes were proposed in this pull request?
   Datetime building should fail if the year, month, ..., second combination is invalid, when ANSI mode is enabled. This patch should update MakeDate, MakeTimestamp and MakeInterval.
   
   ### Why are the changes needed?
   For ANSI mode.
   
   ### Does this PR introduce _any_ user-facing change?
   No.
   
   ### How was this patch tested?
   Added UT and Existing UT.
   


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131983 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131983/testReport)** for PR 30516 at commit [`e87b129`](https://github.com/apache/spark/commit/e87b12910943027b079ead0bf61816cb2910e431).


----------------------------------------------------------------
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] srowen commented on a change in pull request #30516: [SPARK-33498][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
##########
@@ -1776,23 +1776,29 @@ case class MakeDate(year: Expression, month: Expression, day: Expression)
   override def dataType: DataType = DateType
   override def nullable: Boolean = true
 
+  val ansiEnabled: Boolean = SQLConf.get.ansiEnabled
+
   override def nullSafeEval(year: Any, month: Any, day: Any): Any = {
     try {
       val ld = LocalDate.of(year.asInstanceOf[Int], month.asInstanceOf[Int], day.asInstanceOf[Int])
       localDateToDays(ld)
     } catch {
-      case _: java.time.DateTimeException => null
+      case e: java.time.DateTimeException =>

Review comment:
       No big deal but I suppose this could be just ...
   ```
     case e: java.time.DateTimeException if !ansiEnabled => null
   ```




----------------------------------------------------------------
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 #30516: [SPARK-33498][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   Can one of the admins verify this patch?


----------------------------------------------------------------
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] waitinfuture commented on a change in pull request #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
##########
@@ -1023,6 +1023,21 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     checkEvaluation(MakeDate(Literal(2019), Literal(7), Literal(32)), null)
   }
 
+  test("creating values of DateType via make_date with ansiEnabled") {
+    withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") {
+      checkEvaluation(MakeDate(Literal(2013), Literal(7), Literal(15)), Date.valueOf("2013-7-15"))

Review comment:
       done




----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131964 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131964/testReport)** for PR 30516 at commit [`d92fa88`](https://github.com/apache/spark/commit/d92fa88cdefefde58921f555b0711b2453177ccf).


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131974 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131974/testReport)** for PR 30516 at commit [`ab96eb7`](https://github.com/apache/spark/commit/ab96eb701109ef20b46caaa13740a2d662011bd9).


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131983 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131983/testReport)** for PR 30516 at commit [`e87b129`](https://github.com/apache/spark/commit/e87b12910943027b079ead0bf61816cb2910e431).
    * 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 removed a comment on pull request #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131965 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131965/testReport)** for PR 30516 at commit [`547d5b6`](https://github.com/apache/spark/commit/547d5b68118a0ec9cdd49e56e8cba545e8ae8e22).


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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] leanken commented on a change in pull request #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -195,7 +197,7 @@ case class MakeInterval(
   override def inputTypes: Seq[AbstractDataType] = Seq(IntegerType, IntegerType, IntegerType,
     IntegerType, IntegerType, IntegerType, DecimalType(Decimal.MAX_LONG_DIGITS, 6))
   override def dataType: DataType = CalendarIntervalType
-  override def nullable: Boolean = true
+  override def nullable: Boolean = if (failOnError) children.exists(_.nullable) else true

Review comment:
       nullable on other two class should be the same.




----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131980 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131980/testReport)** for PR 30516 at commit [`c6a9aba`](https://github.com/apache/spark/commit/c6a9abae03d86eb9bfdc9349fd1287672e050c33).


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   Can one of the admins verify this patch?


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131974 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131974/testReport)** for PR 30516 at commit [`ab96eb7`](https://github.com/apache/spark/commit/ab96eb701109ef20b46caaa13740a2d662011bd9).
    * 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 commented on pull request #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
##########
@@ -1797,23 +1797,26 @@ case class MakeDate(year: Expression, month: Expression, day: Expression)
   override def dataType: DataType = DateType
   override def nullable: Boolean = true
 
+  val ansiEnabled: Boolean = SQLConf.get.ansiEnabled

Review comment:
       and put it in the constructor parameter list




----------------------------------------------------------------
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] leanken commented on a change in pull request #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
##########
@@ -1797,23 +1797,26 @@ case class MakeDate(year: Expression, month: Expression, day: Expression)
   override def dataType: DataType = DateType
   override def nullable: Boolean = true
 
+  val ansiEnabled: Boolean = SQLConf.get.ansiEnabled

Review comment:
       Nit. how about rename to failOnError to align with the former PR like SPARK-33498.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
##########
@@ -1897,6 +1900,8 @@ case class MakeTimestamp(
   override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
     copy(timeZoneId = Option(timeZoneId))
 
+  val ansiEnabled: Boolean = SQLConf.get.ansiEnabled

Review comment:
       ditto.

##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
##########
@@ -1023,6 +1023,21 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     checkEvaluation(MakeDate(Literal(2019), Literal(7), Literal(32)), null)
   }
 
+  test("creating values of DateType via make_date with ansiEnabled") {
+    withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") {
+      checkEvaluation(MakeDate(Literal(2013), Literal(7), Literal(15)), Date.valueOf("2013-7-15"))

Review comment:
       add comments to distinguish when it is returning null, and when it is excepting exception

##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
##########
@@ -1059,6 +1074,52 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     checkEvaluation(makeTimestampExpr, Timestamp.valueOf("2019-08-12 00:00:58.000001"))
   }
 
+  test("creating values of TimestampType via make_timestamp with ansiEnabled") {

Review comment:
       ditto.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
##########
@@ -1955,6 +1960,7 @@ case class MakeTimestamp(
     val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
     val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName)
     val d = Decimal.getClass.getName.stripSuffix("$")
+    val exceptionCode = if (ansiEnabled) "throw e;" else s"${ev.isNull} = true;"

Review comment:
       ditto.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -197,6 +197,8 @@ case class MakeInterval(
   override def dataType: DataType = CalendarIntervalType
   override def nullable: Boolean = true

Review comment:
       we should update nullable like
   
   if (failOnError) children.exists(_.nullable) else true

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -215,19 +217,20 @@ case class MakeInterval(
         min.asInstanceOf[Int],
         sec.map(_.asInstanceOf[Decimal]).getOrElse(Decimal(0, Decimal.MAX_LONG_DIGITS, 6)))
     } catch {
-      case _: ArithmeticException => null
+      case _: ArithmeticException if !ansiEnabled => null
     }
   }
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
     nullSafeCodeGen(ctx, ev, (year, month, week, day, hour, min, sec) => {
       val iu = IntervalUtils.getClass.getName.stripSuffix("$")
       val secFrac = sec.getOrElse("0")
+      val exceptionCode = if (ansiEnabled) "throw e;" else s"${ev.isNull} = true;"
       s"""
         try {
           ${ev.value} = $iu.makeInterval($year, $month, $week, $day, $hour, $min, $secFrac);
         } catch (java.lang.ArithmeticException e) {
-          ${ev.isNull} = true;
+          $exceptionCode

Review comment:
       nit. how about failOnErrorBranch.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
##########
@@ -1797,23 +1797,26 @@ case class MakeDate(year: Expression, month: Expression, day: Expression)
   override def dataType: DataType = DateType
   override def nullable: Boolean = true
 
+  val ansiEnabled: Boolean = SQLConf.get.ansiEnabled
+
   override def nullSafeEval(year: Any, month: Any, day: Any): Any = {
     try {
       val ld = LocalDate.of(year.asInstanceOf[Int], month.asInstanceOf[Int], day.asInstanceOf[Int])
       localDateToDays(ld)
     } catch {
-      case _: java.time.DateTimeException => null
+      case _: java.time.DateTimeException if !ansiEnabled => null
     }
   }
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
     val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
+    val exceptionCode = if (ansiEnabled) "throw e;" else s"${ev.isNull} = true;"

Review comment:
       ditto

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -215,19 +217,20 @@ case class MakeInterval(
         min.asInstanceOf[Int],
         sec.map(_.asInstanceOf[Decimal]).getOrElse(Decimal(0, Decimal.MAX_LONG_DIGITS, 6)))
     } catch {
-      case _: ArithmeticException => null
+      case _: ArithmeticException if !ansiEnabled => null

Review comment:
       ditto.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
##########
@@ -1978,7 +1984,7 @@ case class MakeTimestamp(
         java.time.Instant instant = ldt.atZone($zoneId).toInstant();
         ${ev.value} = $dtu.instantToMicros(instant);
       } catch (java.time.DateTimeException e) {
-        ${ev.isNull} = true;
+        $exceptionCode

Review comment:
       ditto.

##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
##########
@@ -1023,6 +1023,21 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     checkEvaluation(MakeDate(Literal(2019), Literal(7), Literal(32)), null)
   }
 
+  test("creating values of DateType via make_date with ansiEnabled") {

Review comment:
       nit. rename test name to 
   ANSI mode: creating values of DateType via make_date




----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131987 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131987/testReport)** for PR 30516 at commit [`7da906d`](https://github.com/apache/spark/commit/7da906d7891d9bac496ba713b2f501340c35f381).
    * 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] SparkQA commented on pull request #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131965 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131965/testReport)** for PR 30516 at commit [`547d5b6`](https://github.com/apache/spark/commit/547d5b68118a0ec9cdd49e56e8cba545e8ae8e22).


----------------------------------------------------------------
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] waitinfuture commented on pull request #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   @cloud-fan test passed, ready for merge


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131964 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131964/testReport)** for PR 30516 at commit [`d92fa88`](https://github.com/apache/spark/commit/d92fa88cdefefde58921f555b0711b2453177ccf).


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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] waitinfuture commented on a change in pull request #30516: [SPARK-33498][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
##########
@@ -1776,23 +1776,29 @@ case class MakeDate(year: Expression, month: Expression, day: Expression)
   override def dataType: DataType = DateType
   override def nullable: Boolean = true
 
+  val ansiEnabled: Boolean = SQLConf.get.ansiEnabled
+
   override def nullSafeEval(year: Any, month: Any, day: Any): Any = {
     try {
       val ld = LocalDate.of(year.asInstanceOf[Int], month.asInstanceOf[Int], day.asInstanceOf[Int])
       localDateToDays(ld)
     } catch {
-      case _: java.time.DateTimeException => null
+      case e: java.time.DateTimeException =>

Review comment:
       done




----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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] MaxGekk commented on a change in pull request #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -215,19 +227,20 @@ case class MakeInterval(
         min.asInstanceOf[Int],
         sec.map(_.asInstanceOf[Decimal]).getOrElse(Decimal(0, Decimal.MAX_LONG_DIGITS, 6)))
     } catch {
-      case _: ArithmeticException => null
+      case _: ArithmeticException if !failOnError => null
     }
   }
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
     nullSafeCodeGen(ctx, ev, (year, month, week, day, hour, min, sec) => {
       val iu = IntervalUtils.getClass.getName.stripSuffix("$")
       val secFrac = sec.getOrElse("0")
+      val faileOnErrorBranch = if (failOnError) "throw e;" else s"${ev.isNull} = true;"

Review comment:
       Is `faile` type? `faileOnErrorBranch` -> `failOnErrorBranch`. I will fix this in https://github.com/apache/spark/pull/31951.




-- 
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   ok to test


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   thanks, merging to master!


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
##########
@@ -1023,6 +1023,21 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     checkEvaluation(MakeDate(Literal(2019), Literal(7), Literal(32)), null)
   }
 
+  test("ANSI mode: creating values of DateType via make_date") {

Review comment:
       We can simplify the tests by merging it with `creating values of DateType via make_date`
   ```
   test("creating values of DateType via make_date") {
     Seq(true, false).foreach { ansi =>
        withSQLConf(SQLConf.ANSI_ENABLED.key -> ansi.toString) {
         ...
       }
      }
   
     // ansi test
     checkExceptionInExpression...
     ...
     // non-ansi test
     checkEvaluation(..., null)
     ...
   }
   ```




----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
##########
@@ -1014,49 +1014,98 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
   }
 
   test("creating values of DateType via make_date") {
-    checkEvaluation(MakeDate(Literal(2013), Literal(7), Literal(15)), Date.valueOf("2013-7-15"))
-    checkEvaluation(MakeDate(Literal.create(null, IntegerType), Literal(7), Literal(15)), null)
-    checkEvaluation(MakeDate(Literal(2019), Literal.create(null, IntegerType), Literal(19)), null)
-    checkEvaluation(MakeDate(Literal(2019), Literal(7), Literal.create(null, IntegerType)), null)
-    checkEvaluation(MakeDate(Literal(Int.MaxValue), Literal(13), Literal(19)), null)
-    checkEvaluation(MakeDate(Literal(2019), Literal(13), Literal(19)), null)
-    checkEvaluation(MakeDate(Literal(2019), Literal(7), Literal(32)), null)
+    Seq(true, false).foreach({ ansi =>
+      withSQLConf(SQLConf.ANSI_ENABLED.key -> ansi.toString) {
+        checkEvaluation(MakeDate(Literal(2013), Literal(7), Literal(15)), Date.valueOf("2013-7-15"))
+        checkEvaluation(MakeDate(Literal.create(null, IntegerType), Literal(7), Literal(15)), null)
+        checkEvaluation(MakeDate(Literal(2019), Literal.create(null, IntegerType), Literal(19)),
+          null)
+        checkEvaluation(MakeDate(Literal(2019), Literal(7), Literal.create(null, IntegerType)),
+          null)
+      }
+    })
+
+    // ansi test
+    withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") {
+      checkExceptionInExpression[DateTimeException](MakeDate(Literal(Int.MaxValue), Literal(13),
+        Literal(19)), EmptyRow, "Invalid value for Year")
+      checkExceptionInExpression[DateTimeException](MakeDate(Literal(2019),
+        Literal(13), Literal(19)), EmptyRow, "Invalid value for Month")
+      checkExceptionInExpression[DateTimeException](MakeDate(Literal(2019), Literal(7),
+        Literal(32)), EmptyRow, "Invalid value for Day")
+    }
+
+    // non-ansi test
+    withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") {
+      checkEvaluation(MakeDate(Literal(Int.MaxValue), Literal(13), Literal(19)), null)
+      checkEvaluation(MakeDate(Literal(2019), Literal(13), Literal(19)), null)
+      checkEvaluation(MakeDate(Literal(2019), Literal(7), Literal(32)), null)
+    }
   }
 
   test("creating values of TimestampType via make_timestamp") {
-    var makeTimestampExpr = MakeTimestamp(
-      Literal(2013), Literal(7), Literal(15), Literal(8), Literal(15),
-      Literal(Decimal(BigDecimal(23.5), 8, 6)), Some(Literal(ZoneId.systemDefault().getId)))
     val expected = Timestamp.valueOf("2013-7-15 8:15:23.5")
-    checkEvaluation(makeTimestampExpr, expected)
-    checkEvaluation(makeTimestampExpr.copy(timezone = None), expected)
-
-    checkEvaluation(makeTimestampExpr.copy(year = Literal.create(null, IntegerType)), null)
-    checkEvaluation(makeTimestampExpr.copy(year = Literal(Int.MaxValue)), null)
-
-    checkEvaluation(makeTimestampExpr.copy(month = Literal.create(null, IntegerType)), null)
-    checkEvaluation(makeTimestampExpr.copy(month = Literal(13)), null)
-
-    checkEvaluation(makeTimestampExpr.copy(day = Literal.create(null, IntegerType)), null)
-    checkEvaluation(makeTimestampExpr.copy(day = Literal(32)), null)
 
-    checkEvaluation(makeTimestampExpr.copy(hour = Literal.create(null, IntegerType)), null)
-    checkEvaluation(makeTimestampExpr.copy(hour = Literal(25)), null)
+    Seq(true, false).foreach { ansi =>
+      withSQLConf(SQLConf.ANSI_ENABLED.key -> ansi.toString) {
+        var makeTimestampExpr = MakeTimestamp(
+          Literal(2013), Literal(7), Literal(15), Literal(8), Literal(15),
+          Literal(Decimal(BigDecimal(23.5), 8, 6)), Some(Literal(ZoneId.systemDefault().getId)))
+        checkEvaluation(makeTimestampExpr, expected)
+        checkEvaluation(makeTimestampExpr.copy(year = Literal.create(null, IntegerType)), null)
+        checkEvaluation(makeTimestampExpr.copy(month = Literal.create(null, IntegerType)), null)
+        checkEvaluation(makeTimestampExpr.copy(day = Literal.create(null, IntegerType)), null)
+        checkEvaluation(makeTimestampExpr.copy(hour = Literal.create(null, IntegerType)), null)
+        checkEvaluation(makeTimestampExpr.copy(min = Literal.create(null, IntegerType)), null)
+        checkEvaluation(makeTimestampExpr.copy(sec = Literal.create(null, DecimalType(8, 6))), null)
+        checkEvaluation(makeTimestampExpr.copy(timezone = None), expected)
+
+        Seq(
+          (makeTimestampExpr.copy(year = Literal(Int.MaxValue)), null,
+            "Invalid value for Year"),
+          (makeTimestampExpr.copy(month = Literal(13)), null, "Invalid value for Month"),
+          (makeTimestampExpr.copy(day = Literal(32)), null, "Invalid value for Day"),
+          (makeTimestampExpr.copy(hour = Literal(25)), null, "Invalid value for Hour"),
+          (makeTimestampExpr.copy(min = Literal(65)), null, "Invalid value for Min"),
+          (makeTimestampExpr.copy(sec = Literal(Decimal(
+            BigDecimal(70.0), 8, 6))), null, "Invalid value for Second")
+        ).foreach { entry =>
+          if (ansi) {
+            checkExceptionInExpression[DateTimeException](entry._1, EmptyRow, entry._3)
+          } else {
+            checkEvaluation(entry._1, entry._2)

Review comment:
       `entry._2` is always null. I think we only need `Seq[Tuple2]` not `Seq[Tuple3]`. Here we can just write  `checkEvaluation(entry._1, null)`




----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131965 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131965/testReport)** for PR 30516 at commit [`547d5b6`](https://github.com/apache/spark/commit/547d5b68118a0ec9cdd49e56e8cba545e8ae8e22).
    * 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 commented on pull request #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
##########
@@ -1797,23 +1797,26 @@ case class MakeDate(year: Expression, month: Expression, day: Expression)
   override def dataType: DataType = DateType
   override def nullable: Boolean = true
 
+  val ansiEnabled: Boolean = SQLConf.get.ansiEnabled

Review comment:
       and put it to the parameter list




----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131980 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131980/testReport)** for PR 30516 at commit [`c6a9aba`](https://github.com/apache/spark/commit/c6a9abae03d86eb9bfdc9349fd1287672e050c33).
    * 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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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] waitinfuture commented on a change in pull request #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
##########
@@ -1023,6 +1023,21 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     checkEvaluation(MakeDate(Literal(2019), Literal(7), Literal(32)), null)
   }
 
+  test("creating values of DateType via make_date with ansiEnabled") {

Review comment:
       Since MakeDate is NullIntolerant, it returns null when any input is null.




----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   cc @cloud-fan, @leanken 


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131974 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131974/testReport)** for PR 30516 at commit [`ab96eb7`](https://github.com/apache/spark/commit/ab96eb701109ef20b46caaa13740a2d662011bd9).


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131983 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131983/testReport)** for PR 30516 at commit [`e87b129`](https://github.com/apache/spark/commit/e87b12910943027b079ead0bf61816cb2910e431).


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131987 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131987/testReport)** for PR 30516 at commit [`7da906d`](https://github.com/apache/spark/commit/7da906d7891d9bac496ba713b2f501340c35f381).


----------------------------------------------------------------
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] MaxGekk commented on a change in pull request #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -215,19 +227,20 @@ case class MakeInterval(
         min.asInstanceOf[Int],
         sec.map(_.asInstanceOf[Decimal]).getOrElse(Decimal(0, Decimal.MAX_LONG_DIGITS, 6)))
     } catch {
-      case _: ArithmeticException => null
+      case _: ArithmeticException if !failOnError => null
     }
   }
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
     nullSafeCodeGen(ctx, ev, (year, month, week, day, hour, min, sec) => {
       val iu = IntervalUtils.getClass.getName.stripSuffix("$")
       val secFrac = sec.getOrElse("0")
+      val faileOnErrorBranch = if (failOnError) "throw e;" else s"${ev.isNull} = true;"

Review comment:
       Is `faile` typo? `faileOnErrorBranch` -> `failOnErrorBranch`. I will fix this in https://github.com/apache/spark/pull/31951.




-- 
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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


   **[Test build #131987 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131987/testReport)** for PR 30516 at commit [`7da906d`](https://github.com/apache/spark/commit/7da906d7891d9bac496ba713b2f501340c35f381).


----------------------------------------------------------------
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 #30516: [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid

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






----------------------------------------------------------------
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